From 2bdf276687a0b6769f330b993959966239c96936 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 29 Jul 2020 19:20:17 +0300 Subject: [PATCH 001/374] fix access to redis dictionary after connection was dropped once --- src/Dictionaries/RedisDictionarySource.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 124da141899..5215f5035cd 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -139,6 +139,9 @@ namespace DB BlockInputStreamPtr RedisDictionarySource::loadAll() { + if (!client->isConnected()) + client->connect(host, port); + RedisCommand command_for_keys("KEYS"); command_for_keys << "*"; @@ -189,6 +192,9 @@ namespace DB BlockInputStreamPtr RedisDictionarySource::loadIds(const std::vector & ids) { + if (!client->isConnected()) + client->connect(host, port); + if (storage_type != RedisStorageType::SIMPLE) throw Exception{"Cannot use loadIds with \'simple\' storage type", ErrorCodes::UNSUPPORTED_METHOD}; From 37499f2fe17a35f47683f35a4d55fc8601d91582 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 31 Jul 2020 02:06:01 +0300 Subject: [PATCH 002/374] bump CI --- src/Dictionaries/RedisDictionarySource.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 5215f5035cd..8794f0620e2 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -179,6 +179,7 @@ namespace DB primary_with_secondary.addRedisType(key); } } + if (primary_with_secondary.size() > 1) hkeys.add(std::move(primary_with_secondary)); } From 2ae94f4570596224d3702d2ec06cef771add4827 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 31 Jul 2020 19:54:54 +0300 Subject: [PATCH 003/374] Refactor Pipes part 1. --- src/DataStreams/narrowBlockInputStreams.cpp | 24 +++- src/Processors/Pipe.cpp | 143 ++++++++++++++++++++ src/Processors/Pipe.h | 75 +++++++++- src/Processors/QueryPipeline.h | 4 - 4 files changed, 234 insertions(+), 12 deletions(-) diff --git a/src/DataStreams/narrowBlockInputStreams.cpp b/src/DataStreams/narrowBlockInputStreams.cpp index 8464adb5bb8..55460517232 100644 --- a/src/DataStreams/narrowBlockInputStreams.cpp +++ b/src/DataStreams/narrowBlockInputStreams.cpp @@ -29,23 +29,33 @@ Pipes narrowPipes(Pipes pipes, size_t width) if (size <= width) return pipes; - std::vector partitions(width); + std::vector> partitions(width); auto distribution = getDistribution(size, width); for (size_t i = 0; i < size; ++i) - partitions[distribution[i]].emplace_back(std::move(pipes[i])); + partitions[distribution[i]].emplace_back(pipes.getOutputPort(i)); - Pipes res; - res.reserve(width); + Processors concats; + concats.reserve(width); for (size_t i = 0; i < width; ++i) { - auto processor = std::make_shared(partitions[i].at(0).getHeader(), partitions[i].size()); - res.emplace_back(std::move(partitions[i]), std::move(processor)); + auto concat = std::make_shared(partitions[i].at(0)->getHeader(), partitions[i].size()); + size_t next_port = 0; + for (auto & port : concat->getInputs()) + { + connect(*partitions[i][next_port], port); + ++next_port; + } + + concats.emplace_back(std::move(concat)); } - return res; + auto processors = Pipes::detachProcessors(std::move(pipes)); + processors.insert(processors.end(), concats.begin(), concats.end()); + + return Pipes(std::move(processors)); } } diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index b18bb139215..a7e31876b1d 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -54,6 +54,149 @@ static void checkSource(const IProcessor & source) toString(source.getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); } +Pipes::Pipes(ProcessorPtr source) +{ + checkSource(*source); + output_ports.push_back(&source->getOutputs().front()); + header = output_ports.front()->getHeader(); + processors.emplace_back(std::move(source)); + max_parallel_streams = 1; +} + +Pipes::Pipes(Processors processors_) : processors(std::move(processors_)) +{ + /// Create hash table with processors. + std::unordered_set set; + for (const auto & processor : processors) + set.emplace(processor.get()); + + for (auto & processor : processors) + { + for (const auto & port : processor->getInputs()) + { + if (!port.isConnected()) + throw Exception("Cannot create Pipes because processor " + processor->getName() + + " has not connected input port", ErrorCodes::LOGICAL_ERROR); + + const auto * connected_processor = &port.getOutputPort().getProcessor(); + if (set.count(connected_processor) == 0) + throw Exception("Cannot create Pipes because processor " + processor->getName() + + " has input port which is connected with unknown processor " + + connected_processor->getName(), ErrorCodes::LOGICAL_ERROR); + } + + for (auto & port : processor->getOutputs()) + { + if (!port.isConnected()) + { + output_ports.push_back(&port); + continue; + } + + const auto * connected_processor = &port.getInputPort().getProcessor(); + if (set.count(connected_processor) == 0) + throw Exception("Cannot create Pipes because processor " + processor->getName() + + " has output port which is connected with unknown processor " + + connected_processor->getName(), ErrorCodes::LOGICAL_ERROR); + } + } + + if (output_ports.empty()) + throw Exception("Cannot create Pipes because processors don't have any not-connected output ports", + ErrorCodes::LOGICAL_ERROR); + + header = output_ports.front()->getHeader(); + for (size_t i = 1; i < output_ports.size(); ++i) + assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipes"); + + max_parallel_streams = output_ports.size(); +} + +void Pipes::addTransform(ProcessorPtr transform) +{ + auto & inputs = transform->getInputs(); + if (inputs.size() != output_ports.size()) + throw Exception("Cannot add transform " + transform->getName() + " to Pipes because " + "Processor has " + std::to_string(inputs.size()) + " input ports, " + "but " + std::to_string(output_ports.size()) + " expected", ErrorCodes::LOGICAL_ERROR); + + size_t next_output = 0; + for (auto & input : inputs) + { + connect(*output_ports[next_output], input); + ++next_output; + } + + auto & outputs = transform->getOutputs(); + if (outputs.empty()) + throw Exception("Cannot add transform " + transform->getName() + " to Pipes because it has no outputs", + ErrorCodes::LOGICAL_ERROR); + + output_ports.clear(); + output_ports.reserve(outputs.size()); + + for (auto & output : outputs) + output_ports.emplace_back(&output); + + header = output_ports.front()->getHeader(); + for (size_t i = 1; i < output_ports.size(); ++i) + assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipes"); + + if (totals_port) + assertBlocksHaveEqualStructure(header, totals_port->getHeader(), "Pipes"); + + if (extremes_port) + assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); +} + +void Pipes::addSimpleTransform(const ProcessorGetter & getter) +{ + Block new_header; + + auto add_transform = [&](OutputPort *& port, StreamType stream_type) + { + if (!port) + return; + + auto transform = getter(port->getHeader(), stream_type); + + if (transform) + { + if (transform->getInputs().size() != 1) + throw Exception("Processor for query pipeline transform should have single input, " + "but " + transform->getName() + " has " + + toString(transform->getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR); + + if (transform->getOutputs().size() != 1) + throw Exception("Processor for query pipeline transform should have single output, " + "but " + transform->getName() + " has " + + toString(transform->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); + } + + const auto & out_header = transform ? transform->getOutputs().front().getHeader() + : port->getHeader(); + + if (new_header) + assertBlocksHaveEqualStructure(new_header, out_header, "QueryPipeline"); + else + new_header = out_header; + + if (transform) + { + connect(*port, transform->getInputs().front()); + port = &transform->getOutputs().front(); + processors.emplace_back(std::move(transform)); + } + }; + + for (auto & port : output_ports) + add_transform(port, StreamType::Main); + + add_transform(totals_port, StreamType::Totals); + add_transform(extremes_port, StreamType::Extremes); + + header = std::move(new_header); +} Pipe::Pipe(ProcessorPtr source) { diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 085016c3588..ade93a9ed06 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -6,11 +6,84 @@ namespace DB { class Pipe; -using Pipes = std::vector; class IStorage; using StoragePtr = std::shared_ptr; +/// Pipes is a set of processors which represents the part of pipeline. +/// Pipes contains a list of output ports, with specified port for totals and specified port for extremes. +/// All output ports have same header. +/// All other ports are connected, all connections are inside processors set. +class Pipes +{ +public: + /// Create from source. Source must have no input ports and single output. + explicit Pipes(ProcessorPtr source); + /// Create from processors. Use all not-connected output ports as output_ports. Check invariants. + explicit Pipes(Processors processors_); + + Pipes(const Pipes & other) = delete; + Pipes(Pipes && other) = default; + Pipes & operator=(const Pipes & other) = delete; + Pipes & operator=(Pipes && other) = default; + + const Block & getHeader() const { return header; } + bool empty() const { return output_ports.empty(); } + size_t size() const { return output_ports.size(); } + OutputPort * getOutputPort(size_t pos) const { return output_ports[pos]; } + OutputPort * getTotalsPort() const { return totals_port; } + OutputPort * getExtremesPort() const { return extremes_port; } + + /// Add processor to list, add it output ports to output_ports. + /// Processor shouldn't have input ports, output ports shouldn't be connected. + /// Output headers should have same structure and be compatible with current header (if not empty()). + /// void addSource(ProcessorPtr source); + + /// Add processor to list. It should have size() input ports with compatible header. + /// Output ports should have same headers. + /// If totals or extremes are not empty, transform shouldn't change header. + void addTransform(ProcessorPtr transform); + + enum class StreamType + { + Main = 0, /// Stream for query data. There may be several streams of this type. + Totals, /// Stream for totals. No more then one. + Extremes, /// Stream for extremes. No more then one. + }; + + using ProcessorGetter = std::function; + + /// Add transform with single input and single output for each port. + void addSimpleTransform(const ProcessorGetter & port); + + /// Destroy pipes and get processors. + static Processors detachProcessors(Pipes pipes) { return std::move(pipes.processors); } + +private: + Processors processors; + + /// Header is common for all output below. + Block header; + + /// Output ports. Totals and extremes are allowed to be empty. + std::vector output_ports; + OutputPort * totals_port = nullptr; + OutputPort * extremes_port = nullptr; + + /// It is the max number of processors which can be executed in parallel for each step. See QueryPipeline::Streams. + /// Usually, it's the same as the number of output ports. + size_t max_parallel_streams = 0; + + std::vector table_locks; + + /// Some processors may implicitly use Context or temporary Storage created by Interpreter. + /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, + /// because QueryPipeline is alive until query is finished. + std::vector> interpreter_context; + std::vector storage_holders; +}; + + /// Pipe is a set of processors which represents the part of pipeline with single output. /// All processors in pipe are connected. All ports are connected except the output one. class Pipe diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 7bd16ff62fd..a2f00a2343d 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -151,9 +151,6 @@ public: /// Will read from this stream after all data was read from other streams. void addDelayedStream(ProcessorPtr source); - /// Check if resize transform was used. (In that case another distinct transform will be added). - bool hasMixedStreams() const { return has_resize || hasMoreThanOneStream(); } - /// Changes the number of input ports if needed. Adds ResizeTransform. void resize(size_t num_streams, bool force = false, bool strict = false); @@ -167,7 +164,6 @@ public: size_t getNumStreams() const { return streams.size(); } - bool hasMoreThanOneStream() const { return getNumStreams() > 1; } bool hasTotals() const { return totals_having_port != nullptr; } const Block & getHeader() const { return current_header; } From e411916bde096c69b4b95b7dc6b43f57f65d5623 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 3 Aug 2020 14:33:11 +0300 Subject: [PATCH 004/374] Refactor Pipe [part 1]. --- src/DataStreams/narrowBlockInputStreams.cpp | 45 +-- src/DataStreams/narrowBlockInputStreams.h | 3 +- .../ClusterProxy/SelectStreamFactory.cpp | 8 +- .../ClusterProxy/SelectStreamFactory.h | 2 +- .../ClusterProxy/executeQuery.cpp | 2 +- src/Interpreters/ClusterProxy/executeQuery.h | 3 +- src/Processors/Pipe.cpp | 310 +++++++++++++++++- src/Processors/Pipe.h | 47 ++- src/Processors/QueryPipeline.cpp | 8 +- src/Processors/QueryPipeline.h | 4 +- src/Processors/Sources/RemoteSource.cpp | 12 +- src/Storages/IStorage.cpp | 22 ++ src/Storages/IStorage.h | 21 +- src/Storages/Kafka/StorageKafka.cpp | 4 +- src/Storages/Kafka/StorageKafka.h | 2 +- src/Storages/LiveView/StorageLiveView.cpp | 6 +- src/Storages/LiveView/StorageLiveView.h | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 241 +++++++------- .../MergeTree/MergeTreeDataSelectExecutor.h | 10 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 2 +- src/Storages/PartitionCommands.h | 1 - 21 files changed, 531 insertions(+), 224 deletions(-) diff --git a/src/DataStreams/narrowBlockInputStreams.cpp b/src/DataStreams/narrowBlockInputStreams.cpp index 55460517232..17e44e38293 100644 --- a/src/DataStreams/narrowBlockInputStreams.cpp +++ b/src/DataStreams/narrowBlockInputStreams.cpp @@ -23,39 +23,40 @@ namespace } } -Pipes narrowPipes(Pipes pipes, size_t width) +void narrowPipe(Pipe & pipe, size_t width) { - size_t size = pipes.size(); + size_t size = pipe.numOutputPorts(); if (size <= width) - return pipes; + return; std::vector> partitions(width); auto distribution = getDistribution(size, width); - for (size_t i = 0; i < size; ++i) - partitions[distribution[i]].emplace_back(pipes.getOutputPort(i)); - - Processors concats; - concats.reserve(width); - - for (size_t i = 0; i < width; ++i) + pipe.transform([&](OutputPortRawPtrs ports) { - auto concat = std::make_shared(partitions[i].at(0)->getHeader(), partitions[i].size()); - size_t next_port = 0; - for (auto & port : concat->getInputs()) + for (size_t i = 0; i < size; ++i) + partitions[distribution[i]].emplace_back(ports[i]); + + Processors concats; + concats.reserve(width); + + for (size_t i = 0; i < width; ++i) { - connect(*partitions[i][next_port], port); - ++next_port; + auto concat = std::make_shared(partitions[i].at(0)->getHeader(), + partitions[i].size()); + size_t next_port = 0; + for (auto & port : concat->getInputs()) + { + connect(*partitions[i][next_port], port); + ++next_port; + } + + concats.emplace_back(std::move(concat)); } - concats.emplace_back(std::move(concat)); - } - - auto processors = Pipes::detachProcessors(std::move(pipes)); - processors.insert(processors.end(), concats.begin(), concats.end()); - - return Pipes(std::move(processors)); + return concats; + }); } } diff --git a/src/DataStreams/narrowBlockInputStreams.h b/src/DataStreams/narrowBlockInputStreams.h index 0477d72b3de..97e9c164ddc 100644 --- a/src/DataStreams/narrowBlockInputStreams.h +++ b/src/DataStreams/narrowBlockInputStreams.h @@ -7,7 +7,6 @@ namespace DB { class Pipe; -using Pipes = std::vector; /** If the number of sources of `inputs` is greater than `width`, * then glues the sources to each other (using ConcatBlockInputStream), @@ -16,6 +15,6 @@ using Pipes = std::vector; * Trying to glue the sources with each other uniformly randomly. * (to avoid overweighting if the distribution of the amount of data in different sources is subject to some pattern) */ -Pipes narrowPipes(Pipes pipes, size_t width); +void narrowPipe(Pipe & pipe, size_t width); } diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 51f7e93552a..97d3c71019b 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -113,7 +113,7 @@ void SelectStreamFactory::createForShard( const String &, const ASTPtr & query_ast, const Context & context, const ThrottlerPtr & throttler, const SelectQueryInfo &, - Pipes & res) + Pipes & pipes) { bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; bool add_totals = false; @@ -130,7 +130,7 @@ void SelectStreamFactory::createForShard( auto emplace_local_stream = [&]() { - res.emplace_back(createLocalStream(modified_query_ast, header, context, processed_stage).getPipe()); + pipes.emplace_back(createLocalStream(modified_query_ast, header, context, processed_stage).getPipe()); }; String modified_query = formattedAST(modified_query_ast); @@ -143,7 +143,7 @@ void SelectStreamFactory::createForShard( if (!table_func_ptr) remote_query_executor->setMainTable(main_table); - res.emplace_back(createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes)); + pipes.emplace_back(createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes)); }; const auto & settings = context.getSettingsRef(); @@ -285,7 +285,7 @@ void SelectStreamFactory::createForShard( } }; - res.emplace_back(createDelayedPipe(header, lazily_create_stream)); + pipes.emplace_back(createDelayedPipe(header, lazily_create_stream)); } else emplace_remote_stream(); diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index da1d18b6dd9..80f72fd0024 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -39,7 +39,7 @@ public: const String & query, const ASTPtr & query_ast, const Context & context, const ThrottlerPtr & throttler, const SelectQueryInfo & query_info, - Pipes & res) override; + Pipes & pipes) override; private: const Block header; diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 451a8873f41..d7495a8e0c0 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -74,7 +74,7 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin return new_context; } -Pipes executeQuery( +Pipe executeQuery( IStreamFactory & stream_factory, const ClusterPtr & cluster, Poco::Logger * log, const ASTPtr & query_ast, const Context & context, const Settings & settings, const SelectQueryInfo & query_info) { diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index dcbbe0c7e95..f0d9539770d 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -12,7 +12,6 @@ class Cluster; struct SelectQueryInfo; class Pipe; -using Pipes = std::vector; namespace ClusterProxy { @@ -26,7 +25,7 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin /// Execute a distributed query, creating a vector of BlockInputStreams, from which the result can be read. /// `stream_factory` object encapsulates the logic of creating streams for a different type of query /// (currently SELECT, DESCRIBE). -Pipes executeQuery( +Pipe executeQuery( IStreamFactory & stream_factory, const ClusterPtr & cluster, Poco::Logger * log, const ASTPtr & query_ast, const Context & context, const Settings & settings, const SelectQueryInfo & query_info); diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index a7e31876b1d..b5ca7f24654 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -1,6 +1,11 @@ #include #include #include +#include +#include +#include +#include +#include namespace DB { @@ -54,16 +59,88 @@ static void checkSource(const IProcessor & source) toString(source.getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); } -Pipes::Pipes(ProcessorPtr source) +static OutputPort * uniteExtremes(const OutputPortRawPtrs & ports, const Block & header, Processors & processors) { - checkSource(*source); + if (ports.empty()) + return nullptr; + + if (ports.size() == 1) + return ports.front(); + + /// Here we calculate extremes for extremes in case we unite several pipelines. + /// Example: select number from numbers(2) union all select number from numbers(3) + + /// ->> Resize -> Extremes --(output port)----> Empty + /// --(extremes port)--> ... + + auto resize = std::make_shared(header, ports.size(), 1); + auto extremes = std::make_shared(header); + auto sink = std::make_shared(header); + + auto * extremes_port = &extremes->getExtremesPort(); + + auto in = resize->getInputs().begin(); + for (const auto & port : ports) + connect(*port, *(in++)); + + connect(resize->getOutputs().front(), extremes->getInputPort()); + connect(extremes->getOutputPort(), sink->getPort()); + + processors.emplace_back(std::move(resize)); + processors.emplace_back(std::move(extremes)); + processors.emplace_back(std::move(sink)); + + return extremes_port; +} + +static OutputPort * uniteTotals(const OutputPortRawPtrs & ports, const Block & header, Processors & processors) +{ + if (ports.empty()) + return nullptr; + + if (ports.size() == 1) + return ports.front(); + + /// Calculate totals fro several streams. + /// Take totals from first sources which has any, skip others. + + /// ->> Concat -> Limit + + auto concat = std::make_shared(header, ports.size()); + auto limit = std::make_shared(header, 1, 0); + + auto * totals_port = &limit->getOutputPort(); + + auto in = concat->getInputs().begin(); + for (const auto & port : ports) + connect(*port, *(in++)); + + connect(concat->getOutputs().front(), limit->getInputPort()); + + processors.emplace_back(std::move(concat)); + processors.emplace_back(std::move(limit)); + + return totals_port; +} + +Pipe::Pipe(ProcessorPtr source) +{ + if (auto * source_from_input_stream = typeid_cast(source.get())) + { + /// Special case for SourceFromInputStream. Will remove it later. + totals_port = source_from_input_stream->getTotalsPort(); + extremes_port = source_from_input_stream->getExtremesPort(); + } + else if (source->getOutputs().size() != 1) + checkSource(*source); + output_ports.push_back(&source->getOutputs().front()); header = output_ports.front()->getHeader(); processors.emplace_back(std::move(source)); max_parallel_streams = 1; } -Pipes::Pipes(Processors processors_) : processors(std::move(processors_)) +Pipe::Pipe(Processors processors_) : processors(std::move(processors_)) { /// Create hash table with processors. std::unordered_set set; @@ -75,12 +152,12 @@ Pipes::Pipes(Processors processors_) : processors(std::move(processors_)) for (const auto & port : processor->getInputs()) { if (!port.isConnected()) - throw Exception("Cannot create Pipes because processor " + processor->getName() + + throw Exception("Cannot create Pipe because processor " + processor->getName() + " has not connected input port", ErrorCodes::LOGICAL_ERROR); const auto * connected_processor = &port.getOutputPort().getProcessor(); if (set.count(connected_processor) == 0) - throw Exception("Cannot create Pipes because processor " + processor->getName() + + throw Exception("Cannot create Pipe because processor " + processor->getName() + " has input port which is connected with unknown processor " + connected_processor->getName(), ErrorCodes::LOGICAL_ERROR); } @@ -95,25 +172,160 @@ Pipes::Pipes(Processors processors_) : processors(std::move(processors_)) const auto * connected_processor = &port.getInputPort().getProcessor(); if (set.count(connected_processor) == 0) - throw Exception("Cannot create Pipes because processor " + processor->getName() + + throw Exception("Cannot create Pipe because processor " + processor->getName() + " has output port which is connected with unknown processor " + connected_processor->getName(), ErrorCodes::LOGICAL_ERROR); } } if (output_ports.empty()) - throw Exception("Cannot create Pipes because processors don't have any not-connected output ports", + throw Exception("Cannot create Pipe because processors don't have any not-connected output ports", ErrorCodes::LOGICAL_ERROR); header = output_ports.front()->getHeader(); for (size_t i = 1; i < output_ports.size(); ++i) - assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipes"); + assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipe"); max_parallel_streams = output_ports.size(); } -void Pipes::addTransform(ProcessorPtr transform) +static Pipes removeEmptyPipes(Pipes pipes) { + Pipes res; + res.reserve(pipes.size()); + + for (auto & pipe : pipes) + { + if (!pipe.empty()) + res.emplace_back(std::move(pipe)); + } + + return res; +} + +Pipe Pipe::unitePipes(Pipes pipes) +{ + pipes = removeEmptyPipes(std::move(pipes)); + + if (pipes.empty()) + return {}; + + if (pipes.size() == 1) + return std::move(pipes[0]); + + Pipe res; + OutputPortRawPtrs totals; + OutputPortRawPtrs extremes; + res.header = pipes.front().header; + + for (auto & pipe : pipes) + { + assertBlocksHaveEqualStructure(res.header, pipe.header, "Pipe::unitePipes"); + res.processors.insert(res.processors.end(), pipe.processors.begin(), pipe.processors.end()); + res.output_ports.insert(res.output_ports.end(), pipe.output_ports.begin(), pipe.output_ports.end()); + res.table_locks.insert(res.table_locks.end(), pipe.table_locks.begin(), pipe.table_locks.end()); + res.storage_holders.insert(res.storage_holders.end(), pipe.storage_holders.begin(), pipe.storage_holders.end()); + res.interpreter_context.insert(res.interpreter_context.end(), + pipe.interpreter_context.begin(), pipe.interpreter_context.end()); + + res.max_parallel_streams += pipe.max_parallel_streams; + + if (pipe.totals_port) + totals.emplace_back(pipe.totals_port); + + if (pipe.extremes_port) + extremes.emplace_back(pipe.extremes_port); + } + + res.totals_port = uniteTotals(totals, res.header, res.processors); + res.extremes_port = uniteExtremes(extremes, res.header, res.processors); +} + +//void Pipe::addPipes(Pipe pipes) +//{ +// if (processors.empty()) +// { +// *this = std::move(pipes); +// return; +// } +// +// if (pipes.processors.empty()) +// return; +// +// assertBlocksHaveEqualStructure(header, pipes.header, "Pipe"); +// +// max_parallel_streams += pipes.max_parallel_streams; +// processors.insert(processors.end(), pipes.processors.begin(), pipes.processors.end()); +// +// OutputPortRawPtrs totals; +// if (totals_port) +// totals.emplace_back(totals_port); +// if (pipes.totals_port) +// totals.emplace_back(pipes.totals_port); +// if (!totals.empty()) +// totals_port = uniteTotals(totals, header, processors); +// +// OutputPortRawPtrs extremes; +// if (extremes_port) +// extremes.emplace_back(extremes_port); +// if (pipes.extremes_port) +// extremes.emplace_back(pipes.extremes_port); +// if (!extremes.empty()) +// extremes_port = uniteExtremes(extremes, header, processors); +//} + +//void Pipe::addSource(ProcessorPtr source) +//{ +// checkSource(*source); +// const auto & source_header = output_ports.front()->getHeader(); +// +// assertBlocksHaveEqualStructure(header, source_header, "Pipes"); !!!! +// +// output_ports.push_back(&source->getOutputs().front()); +// processors.emplace_back(std::move(source)); +// +// max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); +//} + +void Pipe::addTotalsSource(ProcessorPtr source) +{ + if (output_ports.empty()) + throw Exception("Cannot add totals source to empty Pipe.", ErrorCodes::LOGICAL_ERROR); + + if (totals_port) + throw Exception("Totals source was already added to Pipe.", ErrorCodes::LOGICAL_ERROR); + + checkSource(*source); + const auto & source_header = output_ports.front()->getHeader(); + + assertBlocksHaveEqualStructure(header, source_header, "Pipes"); + + totals_port = &source->getOutputs().front(); + processors.emplace_back(std::move(source)); +} + +void Pipe::addExtremesSource(ProcessorPtr source) +{ + if (output_ports.empty()) + throw Exception("Cannot add extremes source to empty Pipe.", ErrorCodes::LOGICAL_ERROR); + + if (extremes_port) + throw Exception("Extremes source was already added to Pipe.", ErrorCodes::LOGICAL_ERROR); + + checkSource(*source); + const auto & source_header = output_ports.front()->getHeader(); + + assertBlocksHaveEqualStructure(header, source_header, "Pipes"); + + extremes_port = &source->getOutputs().front(); + processors.emplace_back(std::move(source)); +} + +void Pipe::addTransform(ProcessorPtr transform) +{ + if (output_ports.empty()) + throw Exception("Cannot add transform to empty Pipe.", ErrorCodes::LOGICAL_ERROR); + auto & inputs = transform->getInputs(); if (inputs.size() != output_ports.size()) throw Exception("Cannot add transform " + transform->getName() + " to Pipes because " @@ -147,10 +359,15 @@ void Pipes::addTransform(ProcessorPtr transform) if (extremes_port) assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); + + max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } -void Pipes::addSimpleTransform(const ProcessorGetter & getter) +void Pipe::addSimpleTransform(const ProcessorGetter & getter) { + if (output_ports.empty()) + throw Exception("Cannot add simple transform to empty Pipe.", ErrorCodes::LOGICAL_ERROR); + Block new_header; auto add_transform = [&](OutputPort *& port, StreamType stream_type) @@ -198,6 +415,77 @@ void Pipes::addSimpleTransform(const ProcessorGetter & getter) header = std::move(new_header); } +void Pipe::transform(const Transformer & transformer) +{ + if (output_ports.empty()) + throw Exception("Cannot transform empty Pipe.", ErrorCodes::LOGICAL_ERROR); + + auto new_processors = transformer(output_ports); + + /// Create hash table with new processors. + std::unordered_set set; + for (const auto & processor : new_processors) + set.emplace(processor.get()); + + for (const auto & port : output_ports) + { + if (!port->isConnected()) + throw Exception("Transformation of Pipe is not valid because output port (" + + port->getHeader().dumpStructure() + ") is not connected", ErrorCodes::LOGICAL_ERROR); + + set.emplace(&port->getProcessor()); + } + + OutputPortRawPtrs new_output_ports; + for (const auto & processor : new_processors) + { + for (const auto & port : processor->getInputs()) + { + if (!port.isConnected()) + throw Exception("Transformation of Pipe is not valid because processor " + processor->getName() + + " has not connected input port", ErrorCodes::LOGICAL_ERROR); + + const auto * connected_processor = &port.getOutputPort().getProcessor(); + if (set.count(connected_processor) == 0) + throw Exception("Transformation of Pipe is not valid because processor " + processor->getName() + + " has input port which is connected with unknown processor " + + connected_processor->getName(), ErrorCodes::LOGICAL_ERROR); + } + + for (auto & port : processor->getOutputs()) + { + if (!port.isConnected()) + { + new_output_ports.push_back(&port); + continue; + } + + const auto * connected_processor = &port.getInputPort().getProcessor(); + if (set.count(connected_processor) == 0) + throw Exception("Transformation of Pipe is not valid because processor " + processor->getName() + + " has output port which is connected with unknown processor " + + connected_processor->getName(), ErrorCodes::LOGICAL_ERROR); + } + } + + if (output_ports.empty()) + throw Exception("Transformation of Pipe is not valid because processors don't have any " + "not-connected output ports", ErrorCodes::LOGICAL_ERROR); + + header = output_ports.front()->getHeader(); + for (size_t i = 1; i < output_ports.size(); ++i) + assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipe"); + + if (totals_port) + assertBlocksHaveEqualStructure(header, totals_port->getHeader(), "Pipes"); + + if (extremes_port) + assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); + + max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); +} + +/* Pipe::Pipe(ProcessorPtr source) { if (auto * source_from_input_stream = typeid_cast(source.get())) @@ -295,5 +583,5 @@ void Pipe::enableQuota() source->enableQuota(); } } - +*/ } diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index ade93a9ed06..316d971330f 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -6,39 +6,52 @@ namespace DB { class Pipe; +using Pipes = std::vector; class IStorage; using StoragePtr = std::shared_ptr; +using OutputPortRawPtrs = std::vector; + /// Pipes is a set of processors which represents the part of pipeline. /// Pipes contains a list of output ports, with specified port for totals and specified port for extremes. /// All output ports have same header. /// All other ports are connected, all connections are inside processors set. -class Pipes +class Pipe { public: + /// Default constructor creates empty pipe. Generally, you cannot do anything with it except to check it is empty(). + /// You cannot get empty pipe in any other way. All transforms check that result pipe is not empty. + Pipe() = default; /// Create from source. Source must have no input ports and single output. - explicit Pipes(ProcessorPtr source); + explicit Pipe(ProcessorPtr source); /// Create from processors. Use all not-connected output ports as output_ports. Check invariants. - explicit Pipes(Processors processors_); + explicit Pipe(Processors processors_); - Pipes(const Pipes & other) = delete; - Pipes(Pipes && other) = default; - Pipes & operator=(const Pipes & other) = delete; - Pipes & operator=(Pipes && other) = default; + Pipe(const Pipe & other) = delete; + Pipe(Pipe && other) = default; + Pipe & operator=(const Pipe & other) = delete; + Pipe & operator=(Pipe && other) = default; const Block & getHeader() const { return header; } bool empty() const { return output_ports.empty(); } - size_t size() const { return output_ports.size(); } + size_t numOutputPorts() const { return output_ports.size(); } OutputPort * getOutputPort(size_t pos) const { return output_ports[pos]; } OutputPort * getTotalsPort() const { return totals_port; } OutputPort * getExtremesPort() const { return extremes_port; } + /// Add processors form other pipe. It should have same header. + //void addPipes(Pipes pipes); + /// Add processor to list, add it output ports to output_ports. /// Processor shouldn't have input ports, output ports shouldn't be connected. /// Output headers should have same structure and be compatible with current header (if not empty()). /// void addSource(ProcessorPtr source); + /// Add totals and extremes. + void addTotalsSource(ProcessorPtr source); + void addExtremesSource(ProcessorPtr source); + /// Add processor to list. It should have size() input ports with compatible header. /// Output ports should have same headers. /// If totals or extremes are not empty, transform shouldn't change header. @@ -56,8 +69,13 @@ public: /// Add transform with single input and single output for each port. void addSimpleTransform(const ProcessorGetter & port); - /// Destroy pipes and get processors. - static Processors detachProcessors(Pipes pipes) { return std::move(pipes.processors); } + using Transformer = std::function; + + /// Transform Pipe in general way. + void transform(const Transformer & transformer); + + /// Unite several pipes together. They should have same header. + static Pipe unitePipes(Pipes pipes); private: Processors processors; @@ -66,7 +84,7 @@ private: Block header; /// Output ports. Totals and extremes are allowed to be empty. - std::vector output_ports; + OutputPortRawPtrs output_ports; OutputPort * totals_port = nullptr; OutputPort * extremes_port = nullptr; @@ -81,9 +99,12 @@ private: /// because QueryPipeline is alive until query is finished. std::vector> interpreter_context; std::vector storage_holders; + + /// Destroy pipes and get processors. + static Processors detachProcessors(Pipe pipe) { return std::move(pipe.processors); } }; - +/* /// Pipe is a set of processors which represents the part of pipeline with single output. /// All processors in pipe are connected. All ports are connected except the output one. class Pipe @@ -170,5 +191,5 @@ private: friend class QueryPipeline; }; - +*/ } diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index c914eab4e29..1c9ddb722f1 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -793,13 +793,7 @@ void QueryPipeline::initRowsBeforeLimit() Pipe QueryPipeline::getPipe() && { - resize(1); - return std::move(std::move(*this).getPipes()[0]); -} - -Pipes QueryPipeline::getPipes() && -{ - Pipe pipe(processors.detach(), streams.at(0), totals_having_port, extremes_port); + Pipes pipes(processors.detach(), streams.at(0), totals_having_port, extremes_port); pipe.max_parallel_streams = streams.maxParallelStreams(); for (auto & lock : table_locks) diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index a2f00a2343d..f66d2a220cf 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -102,7 +102,6 @@ public: /// All pipes must have same header. void init(Pipes pipes); - void init(Pipe pipe); /// Simple init for single pipe bool initialized() { return !processors.empty(); } bool isCompleted() { return initialized() && streams.empty(); } @@ -197,9 +196,8 @@ public: max_threads = max_threads_; } - /// Convert query pipeline to single or several pipes. + /// Convert query pipeline to pipe. Pipe getPipe() &&; - Pipes getPipes() &&; /// Get internal processors. const Processors & getProcessors() const { return processors.get(); } diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 2f76e0c87d4..13ce2d1f6ee 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -113,18 +113,10 @@ Pipe createRemoteSourcePipe( Pipe pipe(std::make_shared(query_executor, add_aggregation_info)); if (add_totals) - { - auto totals_source = std::make_shared(query_executor); - pipe.setTotalsPort(&totals_source->getPort()); - pipe.addProcessors({std::move(totals_source)}); - } + pipe.addTotalsSource(std::make_shared(query_executor)); if (add_extremes) - { - auto extremes_source = std::make_shared(query_executor); - pipe.setExtremesPort(&extremes_source->getPort()); - pipe.addProcessors({std::move(extremes_source)}); - } + pipe.addExtremesSource(std::make_shared(query_executor)); return pipe; } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index d8f0a0fac7e..3554be67951 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -78,6 +79,27 @@ TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, cons return result; } +Pipe IStorage::read( + const Names & /*column_names*/, + const StorageMetadataPtr & /*metadata_snapshot*/, + const SelectQueryInfo & /*query_info*/, + const Context & /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + unsigned /*num_streams*/) +{ + throw Exception("Method read is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); +} + +Pipe IStorage::alterPartition( + const ASTPtr & /* query */, + const StorageMetadataPtr & /* metadata_snapshot */, + const PartitionCommands & /* commands */, + const Context & /* context */) +{ + throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); +} + void IStorage::alter( const AlterCommands & params, const Context & context, TableLockHolder &) { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 1c75a661339..b67f5b4ef23 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -50,7 +50,6 @@ using ProcessorPtr = std::shared_ptr; using Processors = std::vector; class Pipe; -using Pipes = std::vector; class StoragePolicy; using StoragePolicyPtr = std::shared_ptr; @@ -136,8 +135,6 @@ public: using ColumnSizeByName = std::unordered_map; virtual ColumnSizeByName getColumnSizes() const { return {}; } -public: - /// Get mutable version (snapshot) of storage metadata. Metadata object is /// multiversion, so it can be concurrently chaged, but returned copy can be /// used without any locks. @@ -183,7 +180,7 @@ private: /// Multiversion storage metadata. Allows to read/write storage metadata /// without locks. MultiVersionStorageMetadataPtr metadata; -private: + RWLockImpl::LockHolder tryLockTimed( const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const SettingSeconds & acquire_timeout) const; @@ -276,17 +273,14 @@ public: * changed during lifetime of the returned pipeline, but the snapshot is * guaranteed to be immutable. */ - virtual Pipes read( + virtual Pipe read( const Names & /*column_names*/, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, - unsigned /*num_streams*/) - { - throw Exception("Method read is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); - } + unsigned /*num_streams*/); /** Writes the data to a table. * Receives a description of the query, which can contain information about the data write method. @@ -355,10 +349,11 @@ public: /** ALTER tables with regard to its partitions. * Should handle locks for each command on its own. */ - virtual Pipes alterPartition(const ASTPtr & /* query */, const StorageMetadataPtr & /* metadata_snapshot */, const PartitionCommands & /* commands */, const Context & /* context */) - { - throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); - } + virtual Pipe alterPartition( + const ASTPtr & /* query */, + const StorageMetadataPtr & /* metadata_snapshot */, + const PartitionCommands & /* commands */, + const Context & /* context */); /// Checks that partition commands can be applied to storage. virtual void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index c0f7ef8e44d..87de9714e73 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -198,7 +198,7 @@ String StorageKafka::getDefaultClientId(const StorageID & table_id_) } -Pipes StorageKafka::read( +Pipe StorageKafka::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /* query_info */, @@ -227,7 +227,7 @@ Pipes StorageKafka::read( } LOG_DEBUG(log, "Starting reading {} streams", pipes.size()); - return pipes; + return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index b7e6ea2a7e0..2986f46ebcc 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -37,7 +37,7 @@ public: void startup() override; void shutdown() override; - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 3a6dd86e63e..54ac5bcc791 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -528,7 +528,7 @@ void StorageLiveView::refresh(const Context & context) } } -Pipes StorageLiveView::read( +Pipe StorageLiveView::read( const Names & /*column_names*/, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, @@ -537,7 +537,6 @@ Pipes StorageLiveView::read( const size_t /*max_block_size*/, const unsigned /*num_streams*/) { - Pipes pipes; { std::lock_guard lock(mutex); if (!(*blocks_ptr)) @@ -545,9 +544,8 @@ Pipes StorageLiveView::read( if (getNewBlocks()) condition.notify_all(); } - pipes.emplace_back(std::make_shared(blocks_ptr, getHeader())); + return Pipe(std::make_shared(blocks_ptr, getHeader())); } - return pipes; } BlockInputStreams StorageLiveView::watch( diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 13386c7a4e6..43afd169a92 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -124,7 +124,7 @@ public: void refresh(const Context & context); - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index acd4602f3a6..da22303a379 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -147,7 +147,7 @@ static RelativeSize convertAbsoluteSampleSizeToRelative(const ASTPtr & node, siz } -Pipes MergeTreeDataSelectExecutor::read( +Pipe MergeTreeDataSelectExecutor::read( const Names & column_names_to_return, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -162,7 +162,7 @@ Pipes MergeTreeDataSelectExecutor::read( max_block_numbers_to_read); } -Pipes MergeTreeDataSelectExecutor::readFromParts( +Pipe MergeTreeDataSelectExecutor::readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names_to_return, const StorageMetadataPtr & metadata_snapshot, @@ -647,7 +647,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( ProfileEvents::increment(ProfileEvents::SelectedRanges, sum_ranges); ProfileEvents::increment(ProfileEvents::SelectedMarks, sum_marks); - Pipes res; + Pipe res; /// Projection, that needed to drop columns, which have appeared by execution /// of some extra expressions, and to allow execute the same expressions later. @@ -721,31 +721,37 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( if (use_sampling) { - for (auto & pipe : res) - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), filter_expression, filter_function->getColumnName(), false)); + res.addSimpleTransform([&filter_expression, &filter_function](const Block & header, Pipe::StreamType) + { + return std::make_shared( + header, filter_expression, filter_function->getColumnName(), false); + }); } if (result_projection) { - for (auto & pipe : res) - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), result_projection)); + res.addSimpleTransform([&result_projection](const Block & header, Pipe::StreamType) + { + return std::make_shared(header, result_projection); + }); } /// By the way, if a distributed query or query to a Merge table is made, then the `_sample_factor` column can have different values. if (sample_factor_column_queried) { - for (auto & pipe : res) - pipe.addSimpleTransform(std::make_shared>( - pipe.getHeader(), std::make_shared(), used_sample_factor, "_sample_factor")); + res.addSimpleTransform([used_sample_factor](const Block & header, Pipe::StreamType) + { + return std::make_shared>( + header, std::make_shared(), used_sample_factor, "_sample_factor"); + }); } if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) { - for (auto & pipe : res) - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), query_info.prewhere_info->remove_columns_actions)); + res.addSimpleTransform([&query_info](const Block & header, Pipe::StreamType) + { + return std::make_shared(header, query_info.prewhere_info->remove_columns_actions); + }); } return res; @@ -774,7 +780,7 @@ size_t roundRowsOrBytesToMarks( } -Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( +Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( RangesInDataParts && parts, size_t num_streams, const Names & column_names, @@ -822,13 +828,13 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; - Pipes res; if (0 == sum_marks) - return res; + return {}; if (num_streams > 1) { /// Parallel query execution. + Pipes res; /// Reduce the number of num_streams if the data is small. if (sum_marks < num_streams * min_marks_for_concurrent_read && parts.size() < num_streams) @@ -867,10 +873,13 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( res.emplace_back(std::move(source)); } + + return Pipe::unitePipes(std::move(res)); } else { /// Sequential query execution. + Pipes res; for (const auto & part : parts) { @@ -882,18 +891,15 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( res.emplace_back(std::move(source)); } + auto pipe = Pipe::unitePipes(std::move(res)); + /// Use ConcatProcessor to concat sources together. /// It is needed to read in parts order (and so in PK order) if single thread is used. - if (res.size() > 1) - { - auto concat = std::make_shared(res.front().getHeader(), res.size()); - Pipe pipe(std::move(res), std::move(concat)); - res = Pipes(); - res.emplace_back(std::move(pipe)); - } - } + if (pipe.numOutputPorts() > 1) + pipe.addTransform(std::make_shared(pipe.getHeader(), pipe.numOutputPorts())); - return res; + return pipe; + } } static ExpressionActionsPtr createProjection(const Pipe & pipe, const MergeTreeData & data) @@ -904,7 +910,7 @@ static ExpressionActionsPtr createProjection(const Pipe & pipe, const MergeTreeD return projection; } -Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( +Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( RangesInDataParts && parts, size_t num_streams, const Names & column_names, @@ -956,7 +962,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( Pipes res; if (sum_marks == 0) - return res; + return {}; /// Let's split ranges to avoid reading much data. auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size](const auto & ranges, int direction) @@ -1101,40 +1107,45 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( reader_settings, virt_columns, part.part_index_in_query)); - - pipes.back().addSimpleTransform(std::make_shared(pipes.back().getHeader())); } } - if (pipes.size() > 1 && need_preliminary_merge) + auto pipe = Pipe::unitePipes(std::move(pipes)); + + if (input_order_info->direction == 1) + { + pipe.addSimpleTransform([](const Block & header, Pipe::StreamType) + { + return std::make_shared(header); + }); + } + + if (pipe.numOutputPorts() > 1 && need_preliminary_merge) { SortDescription sort_description; for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j], - input_order_info->direction, 1); + input_order_info->direction, 1); /// Drop temporary columns, added by 'sorting_key_prefix_expr' - out_projection = createProjection(pipes.back(), data); - for (auto & pipe : pipes) - pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), sorting_key_prefix_expr)); + out_projection = createProjection(pipe, data); + pipe.addSimpleTransform([sorting_key_prefix_expr](const Block & header, Pipe::StreamType) + { + return std::make_shared(header, sorting_key_prefix_expr); + }); - auto merging_sorted = std::make_shared( - pipes.back().getHeader(), pipes.size(), sort_description, max_block_size); + pipe.addTransform(std::make_shared( + pipe.getHeader(), pipe.numOutputPorts(), sort_description, max_block_size)); + } - res.emplace_back(std::move(pipes), std::move(merging_sorted)); - } - else - { - for (auto && pipe : pipes) - res.emplace_back(std::move(pipe)); - } + res.emplace_back(std::move(pipe)); } - return res; + return Pipe::unitePipes(std::move(res)); } -Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( +Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( RangesInDataParts && parts, size_t num_streams, const Names & column_names, @@ -1172,25 +1183,35 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; - Pipes pipes; + Pipe pipe; - for (const auto & part : parts) { - auto source_processor = std::make_shared( - data, metadata_snapshot, part.data_part, max_block_size, settings.preferred_block_size_bytes, - settings.preferred_max_column_in_block_size_bytes, column_names, part.ranges, use_uncompressed_cache, - query_info.prewhere_info, true, reader_settings, - virt_columns, part.part_index_in_query); + Pipes pipes; - Pipe pipe(std::move(source_processor)); - /// Drop temporary columns, added by 'sorting_key_expr' - if (!out_projection) - out_projection = createProjection(pipe, data); + for (const auto & part : parts) + { + auto source_processor = std::make_shared( + data, metadata_snapshot, part.data_part, max_block_size, settings.preferred_block_size_bytes, + settings.preferred_max_column_in_block_size_bytes, column_names, part.ranges, + use_uncompressed_cache, + query_info.prewhere_info, true, reader_settings, + virt_columns, part.part_index_in_query); - pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), metadata_snapshot->getSortingKey().expression)); - pipes.emplace_back(std::move(pipe)); + pipes.emplace_back(std::move(source_processor)); + } + + pipe = Pipe::unitePipes(std::move(pipes)); } + /// Drop temporary columns, added by 'sorting_key_expr' + if (!out_projection) + out_projection = createProjection(pipe, data); + + pipe.addSimpleTransform([&metadata_snapshot](const Block & header, Pipe::StreamType) + { + return std::make_shared(header, metadata_snapshot->getSortingKey().expression); + }); + Names sort_columns = metadata_snapshot->getSortingKeyColumns(); SortDescription sort_description; size_t sort_columns_size = sort_columns.size(); @@ -1198,7 +1219,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names; - Block header = pipes.at(0).getHeader(); + Block header = pipe.getHeader(); for (size_t i = 0; i < sort_columns_size; ++i) sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1); @@ -1208,28 +1229,28 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( { case MergeTreeData::MergingParams::Ordinary: { - return std::make_shared(header, pipes.size(), + return std::make_shared(header, pipe.numOutputPorts(), sort_description, max_block_size); } case MergeTreeData::MergingParams::Collapsing: - return std::make_shared(header, pipes.size(), + return std::make_shared(header, pipe.numOutputPorts(), sort_description, data.merging_params.sign_column, true, max_block_size); case MergeTreeData::MergingParams::Summing: - return std::make_shared(header, pipes.size(), + return std::make_shared(header, pipe.numOutputPorts(), sort_description, data.merging_params.columns_to_sum, partition_key_columns, max_block_size); case MergeTreeData::MergingParams::Aggregating: - return std::make_shared(header, pipes.size(), + return std::make_shared(header, pipe.numOutputPorts(), sort_description, max_block_size); case MergeTreeData::MergingParams::Replacing: - return std::make_shared(header, pipes.size(), + return std::make_shared(header, pipe.numOutputPorts(), sort_description, data.merging_params.version_column, max_block_size); case MergeTreeData::MergingParams::VersionedCollapsing: - return std::make_shared(header, pipes.size(), + return std::make_shared(header, pipe.numOutputPorts(), sort_description, data.merging_params.sign_column, max_block_size); case MergeTreeData::MergingParams::Graphite: @@ -1244,12 +1265,8 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( if (num_streams <= 1 || sort_description.empty()) { - - Pipe pipe(std::move(pipes), get_merging_processor()); - pipes = Pipes(); - pipes.emplace_back(std::move(pipe)); - - return pipes; + pipe.addTransform(get_merging_processor()); + return pipe; } ColumnNumbers key_columns; @@ -1263,63 +1280,47 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( key_columns.emplace_back(desc.column_number); } - Processors selectors; - Processors copiers; - selectors.reserve(pipes.size()); - - for (auto & pipe : pipes) + pipe.addSimpleTransform([&](const Block & header, Pipe::StreamType) { - auto selector = std::make_shared(pipe.getHeader(), num_streams, key_columns); - auto copier = std::make_shared(pipe.getHeader(), num_streams); - connect(pipe.getPort(), selector->getInputPort()); - connect(selector->getOutputPort(), copier->getInputPort()); - selectors.emplace_back(std::move(selector)); - copiers.emplace_back(std::move(copier)); - } + return std::make_shared(header, num_streams, key_columns); + }); - Processors merges; - std::vector input_ports; - merges.reserve(num_streams); - input_ports.reserve(num_streams); - - for (size_t i = 0; i < num_streams; ++i) + pipe.transform([&](OutputPortRawPtrs ports) { - auto merge = get_merging_processor(); - merge->setSelectorPosition(i); - input_ports.emplace_back(merge->getInputs().begin()); - merges.emplace_back(std::move(merge)); - } + Processors processors; + std::vector output_ports; + processors.reserve(ports.size() + num_streams); + output_ports.reserve(ports.size()); - /// Connect outputs of i-th splitter with i-th input port of every merge. - for (auto & resize : copiers) - { - size_t input_num = 0; - for (auto & output : resize->getOutputs()) + for (auto & port : ports) { - connect(output, *input_ports[input_num]); - ++input_ports[input_num]; - ++input_num; + auto copier = std::make_shared(header, num_streams); + connect(*port, copier->getInputPort()); + output_ports.emplace_back(copier->getOutputs().begin()); + processors.emplace_back(std::move(copier)); } - } - Processors processors; - for (auto & pipe : pipes) - { - auto pipe_processors = std::move(pipe).detachProcessors(); - processors.insert(processors.end(), pipe_processors.begin(), pipe_processors.end()); - } + for (size_t i = 0; i < num_streams; ++i) + { + auto merge = get_merging_processor(); + merge->setSelectorPosition(i); + auto input = merge->getInputs().begin(); - pipes.clear(); - pipes.reserve(num_streams); - for (auto & merge : merges) - pipes.emplace_back(&merge->getOutputs().front()); + /// Connect i-th merge with i-th input port of every copier. + for (size_t j = 0; j < ports.size(); ++j) + { + connect(*output_ports[j], *input); + ++output_ports[j]; + ++input; + } - pipes.front().addProcessors(processors); - pipes.front().addProcessors(selectors); - pipes.front().addProcessors(copiers); - pipes.front().addProcessors(merges); + processors.emplace_back(std::move(merge)); + } - return pipes; + return processors; + }); + + return pipe; } /// Calculates a set of mark ranges, that could possibly contain keys, required by condition. diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 9e37d150bd1..5894d6e044b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -24,7 +24,7 @@ public: */ using PartitionIdToMaxBlock = std::unordered_map; - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -33,7 +33,7 @@ public: unsigned num_streams, const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; - Pipes readFromParts( + Pipe readFromParts( MergeTreeData::DataPartsVector parts, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -48,7 +48,7 @@ private: Poco::Logger * log; - Pipes spreadMarkRangesAmongStreams( + Pipe spreadMarkRangesAmongStreams( RangesInDataParts && parts, size_t num_streams, const Names & column_names, @@ -61,7 +61,7 @@ private: const MergeTreeReaderSettings & reader_settings) const; /// out_projection - save projection only with columns, requested to read - Pipes spreadMarkRangesAmongStreamsWithOrder( + Pipe spreadMarkRangesAmongStreamsWithOrder( RangesInDataParts && parts, size_t num_streams, const Names & column_names, @@ -75,7 +75,7 @@ private: const MergeTreeReaderSettings & reader_settings, ExpressionActionsPtr & out_projection) const; - Pipes spreadMarkRangesAmongStreamsFinal( + Pipe spreadMarkRangesAmongStreamsFinal( RangesInDataParts && parts, size_t num_streams, const Names & column_names, diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 0e1555b0ccd..c13f540ad34 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -18,7 +18,7 @@ class StorageFromMergeTreeDataPart final : public ext::shared_ptr_helper; struct PartitionCommand { From 2cca4d5fcf96f2dd90d7ab3b4e73c7c5299c52a3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 3 Aug 2020 16:54:14 +0300 Subject: [PATCH 005/374] Refactor Pipe [part 2]. --- .../ClusterProxy/executeQuery.cpp | 2 +- src/Processors/Pipe.cpp | 7 +- src/Processors/Pipe.h | 10 ++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 22 +++--- src/Storages/PartitionCommands.cpp | 8 +- src/Storages/PartitionCommands.h | 2 +- src/Storages/StorageBuffer.cpp | 79 +++++++++++-------- src/Storages/StorageBuffer.h | 2 +- src/Storages/StorageDictionary.cpp | 7 +- src/Storages/StorageDictionary.h | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageDistributed.h | 2 +- src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageFile.h | 2 +- src/Storages/StorageGenerateRandom.cpp | 4 +- src/Storages/StorageGenerateRandom.h | 2 +- src/Storages/StorageHDFS.cpp | 4 +- src/Storages/StorageHDFS.h | 2 +- src/Storages/StorageInput.cpp | 8 +- src/Storages/StorageInput.h | 2 +- src/Storages/StorageJoin.cpp | 7 +- src/Storages/StorageJoin.h | 2 +- src/Storages/StorageLog.cpp | 4 +- src/Storages/StorageLog.h | 2 +- src/Storages/StorageMaterializedView.cpp | 12 ++- src/Storages/StorageMaterializedView.h | 4 +- src/Storages/StorageMemory.cpp | 4 +- src/Storages/StorageMemory.h | 2 +- src/Storages/StorageMerge.cpp | 77 ++++++++---------- src/Storages/StorageMerge.h | 2 +- src/Storages/StorageMergeTree.cpp | 6 +- src/Storages/StorageMergeTree.h | 4 +- src/Storages/StorageMongoDB.cpp | 7 +- src/Storages/StorageMongoDB.h | 2 +- src/Storages/StorageMySQL.cpp | 7 +- src/Storages/StorageMySQL.h | 2 +- src/Storages/StorageNull.h | 6 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.h | 4 +- src/Storages/StorageS3.cpp | 6 +- src/Storages/StorageS3.h | 2 +- src/Storages/StorageStripeLog.cpp | 7 +- src/Storages/StorageStripeLog.h | 2 +- src/Storages/StorageTinyLog.cpp | 8 +- src/Storages/StorageTinyLog.h | 2 +- src/Storages/StorageURL.cpp | 8 +- src/Storages/StorageURL.h | 2 +- src/Storages/StorageValues.cpp | 8 +- src/Storages/StorageValues.h | 2 +- src/Storages/StorageView.cpp | 11 +-- src/Storages/StorageView.h | 2 +- src/Storages/StorageXDBC.cpp | 3 +- src/Storages/StorageXDBC.h | 2 +- 53 files changed, 184 insertions(+), 212 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index d7495a8e0c0..1ebd3009ff7 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -106,7 +106,7 @@ Pipe executeQuery( for (const auto & shard_info : cluster->getShardsInfo()) stream_factory.createForShard(shard_info, query, query_ast, new_context, throttler, query_info, res); - return res; + return Pipe::unitePipes(std::move(res)); } } diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index b5ca7f24654..4bd282f4802 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -363,7 +363,7 @@ void Pipe::addTransform(ProcessorPtr transform) max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } -void Pipe::addSimpleTransform(const ProcessorGetter & getter) +void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter) { if (output_ports.empty()) throw Exception("Cannot add simple transform to empty Pipe.", ErrorCodes::LOGICAL_ERROR); @@ -415,6 +415,11 @@ void Pipe::addSimpleTransform(const ProcessorGetter & getter) header = std::move(new_header); } +void Pipe::addSimpleTransform(const ProcessorGetter & getter) +{ + addSimpleTransform([&](const Block & stream_header, StreamType) { return getter(stream_header); }); +} + void Pipe::transform(const Transformer & transformer) { if (output_ports.empty()) diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 316d971330f..c6d42f8b269 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -64,10 +64,12 @@ public: Extremes, /// Stream for extremes. No more then one. }; - using ProcessorGetter = std::function; + using ProcessorGetter = std::function; + using ProcessorGetterWithStreamKind = std::function; /// Add transform with single input and single output for each port. void addSimpleTransform(const ProcessorGetter & port); + void addSimpleTransform(const ProcessorGetterWithStreamKind & port); using Transformer = std::function; @@ -77,6 +79,12 @@ public: /// Unite several pipes together. They should have same header. static Pipe unitePipes(Pipes pipes); + /// Do not allow to change the table while the processors of pipe are alive. + void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } + /// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible. + void addInterpreterContext(std::shared_ptr context) { interpreter_context.emplace_back(std::move(context)); } + void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); } + private: Processors processors; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index da22303a379..c4e27a11e7a 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -721,7 +721,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (use_sampling) { - res.addSimpleTransform([&filter_expression, &filter_function](const Block & header, Pipe::StreamType) + res.addSimpleTransform([&filter_expression, &filter_function](const Block & header) { return std::make_shared( header, filter_expression, filter_function->getColumnName(), false); @@ -730,7 +730,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (result_projection) { - res.addSimpleTransform([&result_projection](const Block & header, Pipe::StreamType) + res.addSimpleTransform([&result_projection](const Block & header) { return std::make_shared(header, result_projection); }); @@ -739,7 +739,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( /// By the way, if a distributed query or query to a Merge table is made, then the `_sample_factor` column can have different values. if (sample_factor_column_queried) { - res.addSimpleTransform([used_sample_factor](const Block & header, Pipe::StreamType) + res.addSimpleTransform([used_sample_factor](const Block & header) { return std::make_shared>( header, std::make_shared(), used_sample_factor, "_sample_factor"); @@ -748,7 +748,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) { - res.addSimpleTransform([&query_info](const Block & header, Pipe::StreamType) + res.addSimpleTransform([&query_info](const Block & header) { return std::make_shared(header, query_info.prewhere_info->remove_columns_actions); }); @@ -1114,7 +1114,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( if (input_order_info->direction == 1) { - pipe.addSimpleTransform([](const Block & header, Pipe::StreamType) + pipe.addSimpleTransform([](const Block & header) { return std::make_shared(header); }); @@ -1129,10 +1129,10 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( /// Drop temporary columns, added by 'sorting_key_prefix_expr' out_projection = createProjection(pipe, data); - pipe.addSimpleTransform([sorting_key_prefix_expr](const Block & header, Pipe::StreamType) - { - return std::make_shared(header, sorting_key_prefix_expr); - }); + pipe.addSimpleTransform([sorting_key_prefix_expr](const Block & header) + { + return std::make_shared(header, sorting_key_prefix_expr); + }); pipe.addTransform(std::make_shared( pipe.getHeader(), pipe.numOutputPorts(), sort_description, max_block_size)); @@ -1207,7 +1207,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( if (!out_projection) out_projection = createProjection(pipe, data); - pipe.addSimpleTransform([&metadata_snapshot](const Block & header, Pipe::StreamType) + pipe.addSimpleTransform([&metadata_snapshot](const Block & header) { return std::make_shared(header, metadata_snapshot->getSortingKey().expression); }); @@ -1280,7 +1280,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( key_columns.emplace_back(desc.column_number); } - pipe.addSimpleTransform([&](const Block & header, Pipe::StreamType) + pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, num_streams, key_columns); }); diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index e3f542695cb..991a23ea9a3 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -135,7 +135,7 @@ std::string PartitionCommand::typeToString() const __builtin_unreachable(); } -Pipes convertCommandsResultToSource(const PartitionCommandsResultInfo & commands_result) +Pipe convertCommandsResultToSource(const PartitionCommandsResultInfo & commands_result) { Block header { ColumnWithTypeAndName(std::make_shared(), "command_type"), @@ -180,11 +180,7 @@ Pipes convertCommandsResultToSource(const PartitionCommandsResultInfo & commands } Chunk chunk(std::move(res_columns), commands_result.size()); - - Pipe pipe(std::make_shared(std::move(header), std::move(chunk))); - Pipes result; - result.emplace_back(std::move(pipe)); - return result; + return Pipe(std::make_shared(std::move(header), std::move(chunk))); } } diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index f4d4aebb58c..cea4b38c6a3 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -101,6 +101,6 @@ using PartitionCommandsResultInfo = std::vector; /// used to print info to the user. Tries to create narrowest table for given /// results. For example, if all commands were FREEZE commands, than /// old_part_name column will be absent. -Pipes convertCommandsResultToSource(const PartitionCommandsResultInfo & commands_result); +Pipe convertCommandsResultToSource(const PartitionCommandsResultInfo & commands_result); } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index c4dd94b40a8..4a728841c50 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -146,7 +146,7 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context } -Pipes StorageBuffer::read( +Pipe StorageBuffer::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -155,7 +155,7 @@ Pipes StorageBuffer::read( size_t max_block_size, unsigned num_streams) { - Pipes pipes_from_dst; + Pipe pipe_from_dst; if (destination_id) { @@ -182,7 +182,7 @@ Pipes StorageBuffer::read( query_info.input_order_info = query_info.order_optimizer->getInputOrder(destination, destination_metadata_snapshot); /// The destination table has the same structure of the requested columns and we can simply read blocks from there. - pipes_from_dst = destination->read( + pipe_from_dst = destination->read( column_names, destination_metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } @@ -217,66 +217,75 @@ Pipes StorageBuffer::read( } else { - pipes_from_dst = destination->read( + pipe_from_dst = destination->read( columns_intersection, destination_metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - for (auto & pipe : pipes_from_dst) + pipe_from_dst.addSimpleTransform([&](const Block & header) { - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), header_after_adding_defaults, metadata_snapshot->getColumns().getDefaults(), context)); + return std::make_shared( + header, header_after_adding_defaults, metadata_snapshot->getColumns().getDefaults(), context); + }); - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), header, ConvertingTransform::MatchColumnsMode::Name)); - } + pipe_from_dst.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared( + stream_header, header, ConvertingTransform::MatchColumnsMode::Name); + }); } } - for (auto & pipe : pipes_from_dst) - pipe.addTableLock(destination_lock); + pipe_from_dst.addTableLock(destination_lock); } - Pipes pipes_from_buffers; - pipes_from_buffers.reserve(num_shards); - for (auto & buf : buffers) - pipes_from_buffers.emplace_back(std::make_shared(column_names, buf, *this, metadata_snapshot)); + Pipe pipe_from_buffers; + { + Pipes pipes_from_buffers; + pipes_from_buffers.reserve(num_shards); + for (auto & buf : buffers) + pipes_from_buffers.emplace_back(std::make_shared(column_names, buf, *this, metadata_snapshot)); + + pipe_from_buffers = Pipe::unitePipes(std::move(pipes_from_buffers)); + } /// Convert pipes from table to structure from buffer. - if (!pipes_from_buffers.empty() && !pipes_from_dst.empty() - && !blocksHaveEqualStructure(pipes_from_buffers.front().getHeader(), pipes_from_dst.front().getHeader())) + if (!pipe_from_buffers.empty() && !pipe_from_dst.empty() + && !blocksHaveEqualStructure(pipe_from_buffers.getHeader(), pipe_from_dst.getHeader())) { - for (auto & pipe : pipes_from_dst) - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), - pipes_from_buffers.front().getHeader(), - ConvertingTransform::MatchColumnsMode::Name)); + pipe_from_buffers.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + pipe_from_buffers.getHeader(), + ConvertingTransform::MatchColumnsMode::Name); + }); } /** If the sources from the table were processed before some non-initial stage of query execution, * then sources from the buffers must also be wrapped in the processing pipeline before the same stage. */ if (processed_stage > QueryProcessingStage::FetchColumns) - for (auto & pipe : pipes_from_buffers) - pipe = InterpreterSelectQuery(query_info.query, context, std::move(pipe), SelectQueryOptions(processed_stage)).execute().pipeline.getPipe(); + pipe_from_buffers = InterpreterSelectQuery(query_info.query, context, std::move(pipe_from_buffers), SelectQueryOptions(processed_stage)).execute().pipeline.getPipe(); if (query_info.prewhere_info) { - for (auto & pipe : pipes_from_buffers) - pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), query_info.prewhere_info->prewhere_actions, - query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column)); + pipe_from_buffers.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, query_info.prewhere_info->prewhere_actions, + query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); + }); if (query_info.prewhere_info->alias_actions) { - for (auto & pipe : pipes_from_buffers) - pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), query_info.prewhere_info->alias_actions)); - + pipe_from_buffers.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, query_info.prewhere_info->alias_actions); + }); } } - for (auto & pipe : pipes_from_buffers) - pipes_from_dst.emplace_back(std::move(pipe)); - - return pipes_from_dst; + return Pipe::unitePipes({std::move(pipe_from_dst), std::move(pipe_from_buffers)}); } diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 34625e66a0c..4b7fa763678 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -56,7 +56,7 @@ public: QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const ASTPtr &) const override; - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 4ea028c7ca8..5d92b9cec55 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -123,7 +123,7 @@ void StorageDictionary::checkTableCanBeDropped() const throw Exception("Cannot detach table " + getStorageID().getFullTableName() + " from a database with DICTIONARY engine", ErrorCodes::CANNOT_DETACH_DICTIONARY_AS_TABLE); } -Pipes StorageDictionary::read( +Pipe StorageDictionary::read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, @@ -134,11 +134,8 @@ Pipes StorageDictionary::read( { auto dictionary = context.getExternalDictionariesLoader().getDictionary(dictionary_name); auto stream = dictionary->getBlockInputStream(column_names, max_block_size); - auto source = std::make_shared(stream); /// TODO: update dictionary interface for processors. - Pipes pipes; - pipes.emplace_back(std::move(source)); - return pipes; + return Pipe(std::make_shared(stream)); } diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index f152f8c9932..d822552124d 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -16,7 +16,7 @@ public: void checkTableCanBeDropped() const override; - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 8187b40588e..f536c6ee763 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -467,7 +467,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Con return getQueryProcessingStageImpl(context, to_stage, cluster); } -Pipes StorageDistributed::read( +Pipe StorageDistributed::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 4caf063ce81..b7663676d52 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -70,7 +70,7 @@ public: bool canForceGroupByNoMerge(const Context &, QueryProcessingStage::Enum to_stage, const ASTPtr &) const; QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum to_stage, const ASTPtr &) const override; - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 221e9db9428..471a9f522ab 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -378,7 +378,7 @@ private: }; -Pipes StorageFile::read( +Pipe StorageFile::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, @@ -419,7 +419,7 @@ Pipes StorageFile::read( pipes.emplace_back(std::make_shared( this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults())); - return pipes; + return Pipe::unitePipes(pipes); } diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 2c4a43eb979..ea70dcd5311 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -24,7 +24,7 @@ class StorageFile final : public ext::shared_ptr_helper, public ISt public: std::string getName() const override { return "File"; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 0d5011de4b1..8020c54d8cd 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -438,7 +438,7 @@ void registerStorageGenerateRandom(StorageFactory & factory) }); } -Pipes StorageGenerateRandom::read( +Pipe StorageGenerateRandom::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, @@ -467,7 +467,7 @@ Pipes StorageGenerateRandom::read( for (UInt64 i = 0; i < num_streams; ++i) pipes.emplace_back(std::make_shared(max_block_size, max_array_length, max_string_length, generate(), block_header, context)); - return pipes; + return Pipe::unitePipes(pipes); } } diff --git a/src/Storages/StorageGenerateRandom.h b/src/Storages/StorageGenerateRandom.h index 0d068eb951e..e0f037f9a08 100644 --- a/src/Storages/StorageGenerateRandom.h +++ b/src/Storages/StorageGenerateRandom.h @@ -15,7 +15,7 @@ class StorageGenerateRandom final : public ext::shared_ptr_helper( sources_info, uri_without_path, format_name, compression_method, metadata_snapshot->getSampleBlock(), context_, max_block_size)); - return pipes; + return Pipe::unitePipes(pipes); } BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) diff --git a/src/Storages/StorageHDFS.h b/src/Storages/StorageHDFS.h index 62425cc518f..fdeaf4ae1b3 100644 --- a/src/Storages/StorageHDFS.h +++ b/src/Storages/StorageHDFS.h @@ -19,7 +19,7 @@ class StorageHDFS final : public ext::shared_ptr_helper, public ISt public: String getName() const override { return "HDFS"; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index dc8d7ec1581..5e525210548 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -58,7 +58,7 @@ void StorageInput::setInputStream(BlockInputStreamPtr input_stream_) } -Pipes StorageInput::read( +Pipe StorageInput::read( const Names & /*column_names*/, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, @@ -74,15 +74,13 @@ Pipes StorageInput::read( { /// Send structure to the client. query_context.initializeInput(shared_from_this()); - pipes.emplace_back(std::make_shared(query_context, metadata_snapshot->getSampleBlock())); - return pipes; + return Pipe(std::make_shared(query_context, metadata_snapshot->getSampleBlock())); } if (!input_stream) throw Exception("Input stream is not initialized, input() must be used only in INSERT SELECT query", ErrorCodes::INVALID_USAGE_OF_INPUT); - pipes.emplace_back(std::make_shared(input_stream)); - return pipes; + return Pipe(std::make_shared(input_stream)); } } diff --git a/src/Storages/StorageInput.h b/src/Storages/StorageInput.h index f4425ee8cd5..c19b19e4703 100644 --- a/src/Storages/StorageInput.h +++ b/src/Storages/StorageInput.h @@ -17,7 +17,7 @@ public: /// A table will read from this stream. void setInputStream(BlockInputStreamPtr input_stream_); - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index af1a8bf10d7..32edae20495 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -436,7 +436,7 @@ private: // TODO: multiple stream read and index read -Pipes StorageJoin::read( +Pipe StorageJoin::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, @@ -447,10 +447,7 @@ Pipes StorageJoin::read( { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); - Pipes pipes; - pipes.emplace_back(std::make_shared(*join, max_block_size, metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); - - return pipes; + return Pipe(std::make_shared(*join, max_block_size, metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); } } diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index fb8ffc1c353..386282708c9 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -36,7 +36,7 @@ public: /// Verify that the data structure is suitable for implementing this type of JOIN. void assertCompatible(ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_) const; - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 9cfc906108a..50b31a1f128 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -604,7 +604,7 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount(const StorageMeta return it->second.marks; } -Pipes StorageLog::read( +Pipe StorageLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, @@ -647,7 +647,7 @@ Pipes StorageLog::read( max_read_buffer_size)); } - return pipes; + return Pipe::unitePipes(pipes); } BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 96acb1668e2..49fc9a576c5 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -24,7 +24,7 @@ class StorageLog final : public ext::shared_ptr_helper, public IStor public: String getName() const override { return "Log"; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 5fd1e7792e4..a2e3fae0951 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -106,7 +106,7 @@ QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(cons return getTargetTable()->getQueryProcessingStage(context, to_stage, query_ptr); } -Pipes StorageMaterializedView::read( +Pipe StorageMaterializedView::read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, @@ -122,12 +122,10 @@ Pipes StorageMaterializedView::read( if (query_info.order_optimizer) query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage, metadata_snapshot); - Pipes pipes = storage->read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); + Pipe pipe = storage->read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); + pipe.addTableLock(lock); - for (auto & pipe : pipes) - pipe.addTableLock(lock); - - return pipes; + return pipe; } BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) @@ -250,7 +248,7 @@ void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & command } } -Pipes StorageMaterializedView::alterPartition( +Pipe StorageMaterializedView::alterPartition( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) { checkStatementCanBeForwarded(); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 315d4cf01a2..1ee4246c7f1 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -51,7 +51,7 @@ public: void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override; - Pipes alterPartition(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) override; + Pipe alterPartition(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) override; void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override; @@ -71,7 +71,7 @@ public: ActionLock getActionLock(StorageActionBlockType type) override; - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 3a10b2c7e7d..1c690221967 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -101,7 +101,7 @@ StorageMemory::StorageMemory(const StorageID & table_id_, ColumnsDescription col } -Pipes StorageMemory::read( +Pipe StorageMemory::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, @@ -132,7 +132,7 @@ Pipes StorageMemory::read( pipes.emplace_back(std::make_shared(column_names, begin, end, *this, metadata_snapshot)); } - return pipes; + return Pipe::unitePipes(pipes); } diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 851fe7fc70a..24eee7897bf 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -28,7 +28,7 @@ public: size_t getSize() const { return data.size(); } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index c9858969579..15e9fd65448 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -19,7 +19,6 @@ #include #include #include -#include #include #include #include @@ -128,7 +127,7 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & } -Pipes StorageMerge::read( +Pipe StorageMerge::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -137,7 +136,7 @@ Pipes StorageMerge::read( const size_t max_block_size, unsigned num_streams) { - Pipes res; + Pipe pipe; bool has_table_virtual_column = false; Names real_column_names; @@ -211,22 +210,19 @@ Pipes StorageMerge::read( auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); - auto source_pipes = createSources( + pipe = createSources( storage_metadata_snapshot, query_info, processed_stage, max_block_size, header, table, real_column_names, modified_context, current_streams, has_table_virtual_column); - - for (auto & pipe : source_pipes) - res.emplace_back(std::move(pipe)); } - if (res.empty()) - return res; + if (!pipe.empty()) + narrowPipe(pipe, num_streams); - return narrowPipes(std::move(res), num_streams); + return pipe; } -Pipes StorageMerge::createSources( +Pipe StorageMerge::createSources( const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage, @@ -245,18 +241,17 @@ Pipes StorageMerge::createSources( VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", table_name); - Pipes pipes; + Pipe pipe; if (!storage) { - auto pipe = InterpreterSelectQuery( + pipe = InterpreterSelectQuery( modified_query_info.query, *modified_context, std::make_shared(header), SelectQueryOptions(processed_stage).analyze()).execute().pipeline.getPipe(); pipe.addInterpreterContext(modified_context); - pipes.emplace_back(std::move(pipe)); - return pipes; + return pipe; } auto storage_stage = storage->getQueryProcessingStage(*modified_context, QueryProcessingStage::Complete, query_info.query); @@ -267,7 +262,7 @@ Pipes StorageMerge::createSources( real_column_names.push_back(ExpressionActions::getSmallestColumn(metadata_snapshot->getColumns().getAllPhysical())); - pipes = storage->read(real_column_names, metadata_snapshot, modified_query_info, *modified_context, processed_stage, max_block_size, UInt32(streams_num)); + pipe = storage->read(real_column_names, metadata_snapshot, modified_query_info, *modified_context, processed_stage, max_block_size, UInt32(streams_num)); } else if (processed_stage > storage_stage) { @@ -279,46 +274,39 @@ Pipes StorageMerge::createSources( InterpreterSelectQuery interpreter{modified_query_info.query, *modified_context, SelectQueryOptions(processed_stage)}; - { - Pipe pipe = interpreter.execute().pipeline.getPipe(); - pipes.emplace_back(std::move(pipe)); - } + + pipe = interpreter.execute().pipeline.getPipe(); /** Materialization is needed, since from distributed storage the constants come materialized. * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, * And this is not allowed, since all code is based on the assumption that in the block stream all types are the same. */ - pipes.back().addSimpleTransform(std::make_shared(pipes.back().getHeader())); + pipe.addSimpleTransform([](const Block & header) { return std::make_shared(header); }); } - if (!pipes.empty()) + if (!pipe.empty()) { - if (concat_streams && pipes.size() > 1) - { - auto concat = std::make_shared(pipes.at(0).getHeader(), pipes.size()); - Pipe pipe(std::move(pipes), std::move(concat)); + if (concat_streams && pipe.numOutputPorts() > 1) + pipe.addTransform(std::make_shared(pipe.getHeader(), pipe.numOutputPorts())); - pipes = Pipes(); - pipes.emplace_back(std::move(pipe)); + if (has_table_virtual_column) + { + pipe.addSimpleTransform([name = table_name](const Block & header) + { + return std::make_shared>( + header, std::make_shared(), name, "_table"); + }); } - for (auto & pipe : pipes) - { - if (has_table_virtual_column) - pipe.addSimpleTransform(std::make_shared>( - pipe.getHeader(), std::make_shared(), table_name, "_table")); + /// Subordinary tables could have different but convertible types, like numeric types of different width. + /// We must return streams with structure equals to structure of Merge table. + convertingSourceStream(header, metadata_snapshot, *modified_context, modified_query_info.query, pipe, processed_stage); - /// Subordinary tables could have different but convertible types, like numeric types of different width. - /// We must return streams with structure equals to structure of Merge table. - convertingSourceStream(header, metadata_snapshot, *modified_context, modified_query_info.query, pipe, processed_stage); - - pipe.addTableLock(struct_lock); - pipe.addInterpreterContext(modified_context); - - } + pipe.addTableLock(struct_lock); + pipe.addInterpreterContext(modified_context); } - return pipes; + return pipe; } @@ -452,7 +440,10 @@ void StorageMerge::convertingSourceStream( QueryProcessingStage::Enum processed_stage) { Block before_block_header = pipe.getHeader(); - pipe.addSimpleTransform(std::make_shared(before_block_header, header, ConvertingTransform::MatchColumnsMode::Name)); + pipe.addSimpleTransform([&before_block_header](const Block & header) + { + return std::make_shared(before_block_header, header, ConvertingTransform::MatchColumnsMode::Name); + }); auto where_expression = query->as()->where(); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index f2af25d3f3d..4fa2eaed27d 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -29,7 +29,7 @@ public: QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const ASTPtr &) const override; - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9ef89d62919..898cfa7f766 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -172,7 +172,7 @@ StorageMergeTree::~StorageMergeTree() shutdown(); } -Pipes StorageMergeTree::read( +Pipe StorageMergeTree::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -1052,7 +1052,7 @@ bool StorageMergeTree::optimize( return true; } -Pipes StorageMergeTree::alterPartition( +Pipe StorageMergeTree::alterPartition( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, @@ -1135,7 +1135,7 @@ Pipes StorageMergeTree::alterPartition( if (query_context.getSettingsRef().alter_partition_verbose_result) return convertCommandsResultToSource(result); - return { }; + return {}; } void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, const Context & context) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index e12e646f04e..18fc521b067 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -37,7 +37,7 @@ public: bool supportsIndexForIn() const override { return true; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, @@ -61,7 +61,7 @@ public: bool deduplicate, const Context & context) override; - Pipes alterPartition( + Pipe alterPartition( const ASTPtr & query, const StorageMetadataPtr & /* metadata_snapshot */, const PartitionCommands & commands, diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index ee6296a6c8f..210dc09ba86 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -53,7 +53,7 @@ StorageMongoDB::StorageMongoDB( } -Pipes StorageMongoDB::read( +Pipe StorageMongoDB::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, @@ -79,11 +79,8 @@ Pipes StorageMongoDB::read( sample_block.insert({ column_data.type, column_data.name }); } - Pipes pipes; - pipes.emplace_back(std::make_shared( + return Pipe(std::make_shared( std::make_shared(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size, true))); - - return pipes; } void registerStorageMongoDB(StorageFactory & factory) diff --git a/src/Storages/StorageMongoDB.h b/src/Storages/StorageMongoDB.h index 50dc794f070..a8bd2f4d160 100644 --- a/src/Storages/StorageMongoDB.h +++ b/src/Storages/StorageMongoDB.h @@ -34,7 +34,7 @@ public: std::string getName() const override { return "MongoDB"; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 000fb8030d7..b2e622663c0 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -64,7 +64,7 @@ StorageMySQL::StorageMySQL( } -Pipes StorageMySQL::read( +Pipe StorageMySQL::read( const Names & column_names_, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info_, @@ -94,12 +94,9 @@ Pipes StorageMySQL::read( sample_block.insert({ column_data.type, column_data.name }); } - Pipes pipes; /// TODO: rewrite MySQLBlockInputStream - pipes.emplace_back(std::make_shared( + return Pipe(std::make_shared( std::make_shared(pool.get(), query, sample_block, max_block_size_))); - - return pipes; } diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index 287c65db6f3..a7f98c4379b 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -37,7 +37,7 @@ public: std::string getName() const override { return "MySQL"; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index 072a5e3bc32..226eb05ccf9 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -22,7 +22,7 @@ class StorageNull final : public ext::shared_ptr_helper, public ISt public: std::string getName() const override { return "Null"; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo &, @@ -31,10 +31,8 @@ public: size_t, unsigned) override { - Pipes pipes; - pipes.emplace_back( + return Pipe( std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); - return pipes; } BlockOutputStreamPtr write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &) override diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e2a76f45b82..75bbf5488ff 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3418,7 +3418,7 @@ ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock StorageReplicatedMerg return max_added_blocks; } -Pipes StorageReplicatedMergeTree::read( +Pipe StorageReplicatedMergeTree::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -3864,7 +3864,7 @@ void StorageReplicatedMergeTree::alter( } } -Pipes StorageReplicatedMergeTree::alterPartition( +Pipe StorageReplicatedMergeTree::alterPartition( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 84fbae73fa5..108d4e77d29 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -87,7 +87,7 @@ public: bool supportsReplication() const override { return true; } bool supportsDeduplication() const override { return true; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, @@ -111,7 +111,7 @@ public: void alter(const AlterCommands & params, const Context & query_context, TableLockHolder & table_lock_holder) override; - Pipes alterPartition( + Pipe alterPartition( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index fd27229e530..c08bc4e46ec 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -284,7 +284,7 @@ Strings listFilesWithRegexpMatching(Aws::S3::S3Client & client, const S3::URI & } -Pipes StorageS3::read( +Pipe StorageS3::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, @@ -319,7 +319,9 @@ Pipes StorageS3::read( uri.bucket, key)); - return narrowPipes(std::move(pipes), num_streams); + auto pipe = Pipe::unitePipes(std::move(pipes)); + narrowPipe(pipe, num_streams); + return pipe; } BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 991145139cd..a172d951642 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -41,7 +41,7 @@ public: return name; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index ae8162d5f1b..66d7c79e434 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -290,7 +290,7 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora } -Pipes StorageStripeLog::read( +Pipe StorageStripeLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, @@ -310,8 +310,7 @@ Pipes StorageStripeLog::read( String index_file = table_path + "index.mrk"; if (!disk->exists(index_file)) { - pipes.emplace_back(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); - return pipes; + Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); } CompressedReadBufferFromFile index_in(disk->readFile(index_file, INDEX_BUFFER_SIZE)); @@ -335,7 +334,7 @@ Pipes StorageStripeLog::read( /// We do not keep read lock directly at the time of reading, because we read ranges of data that do not change. - return pipes; + return Pipe::unitePipes(pipes); } diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index dfdf4c381b6..f88120a932e 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -25,7 +25,7 @@ class StorageStripeLog final : public ext::shared_ptr_helper, public: String getName() const override { return "StripeLog"; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index b68ac6ae5f1..e7fe7e2d5f9 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -420,7 +420,7 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const Storage } -Pipes StorageTinyLog::read( +Pipe StorageTinyLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, @@ -431,14 +431,10 @@ Pipes StorageTinyLog::read( { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); - Pipes pipes; - // When reading, we lock the entire storage, because we only have one file // per column and can't modify it concurrently. - pipes.emplace_back(std::make_shared( + return Pipe(std::make_shared( max_block_size, Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)), *this, context.getSettingsRef().max_read_buffer_size)); - - return pipes; } diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index 60dacf6e162..dc6ff101503 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -24,7 +24,7 @@ class StorageTinyLog final : public ext::shared_ptr_helper, publ public: String getName() const override { return "TinyLog"; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 55138bf6650..c2f7bfd18d2 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -13,7 +13,6 @@ #include #include -#include #include #include @@ -178,7 +177,7 @@ std::function IStorageURLBase::getReadPOSTDataCallback( } -Pipes IStorageURLBase::read( +Pipe IStorageURLBase::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -192,8 +191,7 @@ Pipes IStorageURLBase::read( for (const auto & [param, value] : params) request_uri.addQueryParameter(param, value); - Pipes pipes; - pipes.emplace_back(std::make_shared( + return Pipe(std::make_shared( request_uri, getReadMethod(), getReadPOSTDataCallback( @@ -207,8 +205,6 @@ Pipes IStorageURLBase::read( max_block_size, ConnectionTimeouts::getHTTPTimeouts(context), chooseCompressionMethod(request_uri.getPath(), compression_method))); - - return pipes; } BlockOutputStreamPtr IStorageURLBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 69b22e5a1e4..7983ad71520 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -19,7 +19,7 @@ namespace DB class IStorageURLBase : public IStorage { public: - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index d5585edde3b..387d2065f92 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -21,7 +21,7 @@ StorageValues::StorageValues( setInMemoryMetadata(storage_metadata); } -Pipes StorageValues::read( +Pipe StorageValues::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, @@ -32,12 +32,8 @@ Pipes StorageValues::read( { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); - Pipes pipes; - Chunk chunk(res_block.getColumns(), res_block.rows()); - pipes.emplace_back(std::make_shared(res_block.cloneEmpty(), std::move(chunk))); - - return pipes; + return Pipe(std::make_shared(res_block.cloneEmpty(), std::move(chunk))); } } diff --git a/src/Storages/StorageValues.h b/src/Storages/StorageValues.h index 88fb023fb2b..8a1a06eeb54 100644 --- a/src/Storages/StorageValues.h +++ b/src/Storages/StorageValues.h @@ -15,7 +15,7 @@ class StorageValues final : public ext::shared_ptr_helper, public public: std::string getName() const override { return "Values"; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 6e66b770486..2c68be4fdb1 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include @@ -15,12 +14,8 @@ #include #include -#include #include #include -#include -#include - namespace DB { @@ -52,7 +47,7 @@ StorageView::StorageView( } -Pipes StorageView::read( +Pipe StorageView::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -91,9 +86,7 @@ Pipes StorageView::read( column_names, getVirtuals(), getStorageID()), ConvertingTransform::MatchColumnsMode::Name); }); - pipes = std::move(pipeline).getPipes(); - - return pipes; + return std::move(pipeline).getPipe(); } static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query) diff --git a/src/Storages/StorageView.h b/src/Storages/StorageView.h index 9de1f3f0bd8..682c7424b98 100644 --- a/src/Storages/StorageView.h +++ b/src/Storages/StorageView.h @@ -21,7 +21,7 @@ public: bool supportsSampling() const override { return true; } bool supportsFinal() const override { return true; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index fc4bbefe74c..3350a4352db 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include @@ -85,7 +84,7 @@ std::function StorageXDBC::getReadPOSTDataCallback( return [query](std::ostream & os) { os << "query=" << query; }; } -Pipes StorageXDBC::read( +Pipe StorageXDBC::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index 44931af4643..508edf22684 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -15,7 +15,7 @@ namespace DB class StorageXDBC : public IStorageURLBase { public: - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, From d9ca3d8d40880e01f183520fd7540b846220d9fc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 3 Aug 2020 18:54:53 +0300 Subject: [PATCH 006/374] Refactor Pipe [part 3]. --- src/Processors/Pipe.cpp | 105 ++++++++++++------ src/Processors/Pipe.h | 14 ++- src/Processors/QueryPipeline.cpp | 2 +- src/Processors/QueryPipeline.h | 50 +-------- .../QueryPlan/ReadFromStorageStep.cpp | 51 +++++---- 5 files changed, 113 insertions(+), 109 deletions(-) diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index 4bd282f4802..b6c3ecb5f88 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -134,6 +134,9 @@ Pipe::Pipe(ProcessorPtr source) else if (source->getOutputs().size() != 1) checkSource(*source); + if (collected_processors) + collected_processors->emplace_back(source.get()); + output_ports.push_back(&source->getOutputs().front()); header = output_ports.front()->getHeader(); processors.emplace_back(std::move(source)); @@ -187,6 +190,10 @@ Pipe::Pipe(Processors processors_) : processors(std::move(processors_)) assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipe"); max_parallel_streams = output_ports.size(); + + if (collected_processors) + for (const auto & processor : processors) + collected_processors->emplace_back(processor.get()); } static Pipes removeEmptyPipes(Pipes pipes) @@ -204,6 +211,11 @@ static Pipes removeEmptyPipes(Pipes pipes) } Pipe Pipe::unitePipes(Pipes pipes) +{ + return Pipe::unitePipes(std::move(pipes), nullptr); +} + +Pipe Pipe::unitePipes(Pipes pipes, Processors * collected_processors) { pipes = removeEmptyPipes(std::move(pipes)); @@ -217,6 +229,7 @@ Pipe Pipe::unitePipes(Pipes pipes) OutputPortRawPtrs totals; OutputPortRawPtrs extremes; res.header = pipes.front().header; + res.collected_processors = collected_processors; for (auto & pipe : pipes) { @@ -237,8 +250,16 @@ Pipe Pipe::unitePipes(Pipes pipes) extremes.emplace_back(pipe.extremes_port); } + size_t num_processors = res.processors.size(); + res.totals_port = uniteTotals(totals, res.header, res.processors); res.extremes_port = uniteExtremes(extremes, res.header, res.processors); + + if (res.collected_processors) + { + for (; num_processors < res.processors.size(); ++num_processors) + res.collected_processors->emplace_back(res.processors[num_processors]); + } } //void Pipe::addPipes(Pipe pipes) @@ -300,6 +321,9 @@ void Pipe::addTotalsSource(ProcessorPtr source) assertBlocksHaveEqualStructure(header, source_header, "Pipes"); + if (collected_processors) + collected_processors->emplace_back(source.get()); + totals_port = &source->getOutputs().front(); processors.emplace_back(std::move(source)); } @@ -317,6 +341,9 @@ void Pipe::addExtremesSource(ProcessorPtr source) assertBlocksHaveEqualStructure(header, source_header, "Pipes"); + if (collected_processors) + collected_processors->emplace_back(source.get()); + extremes_port = &source->getOutputs().front(); processors.emplace_back(std::move(source)); } @@ -360,6 +387,11 @@ void Pipe::addTransform(ProcessorPtr transform) if (extremes_port) assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); + if (collected_processors) + collected_processors->emplace_back(transform.get()); + + processors.emplace_back(std::move(transform)); + max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } @@ -402,6 +434,10 @@ void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter) { connect(*port, transform->getInputs().front()); port = &transform->getOutputs().front(); + + if (collected_processors) + collected_processors->emplace_back(transform.get()); + processors.emplace_back(std::move(transform)); } }; @@ -487,9 +523,44 @@ void Pipe::transform(const Transformer & transformer) if (extremes_port) assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); + if (collected_processors) + { + for (const auto & processor : processors) + collected_processors->emplace_back(processor.get()); + } + + processors.insert(processors.end(), new_processors.begin(), new_processors.end()); + max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } +void Pipe::setLimits(const ISourceWithProgress::LocalLimits & limits) +{ + for (auto & processor : processors) + { + if (auto * source_with_progress = dynamic_cast(processor.get())) + source_with_progress->setLimits(limits); + } +} + +void Pipe::setQuota(const std::shared_ptr & quota) +{ + for (auto & processor : processors) + { + if (auto * source_with_progress = dynamic_cast(processor.get())) + source_with_progress->setQuota(quota); + } +} + +void Pipe::enableQuota() +{ + for (auto & processor : processors) + { + if (auto * source = dynamic_cast(processor.get())) + source->enableQuota(); + } +} + /* Pipe::Pipe(ProcessorPtr source) { @@ -553,40 +624,6 @@ void Pipe::addSimpleTransform(ProcessorPtr transform) processors.emplace_back(std::move(transform)); } -void Pipe::setLimits(const ISourceWithProgress::LocalLimits & limits) -{ - for (auto & processor : processors) - { - if (auto * source_with_progress = dynamic_cast(processor.get())) - source_with_progress->setLimits(limits); - } -} -void Pipe::setQuota(const std::shared_ptr & quota) -{ - for (auto & processor : processors) - { - if (auto * source_with_progress = dynamic_cast(processor.get())) - source_with_progress->setQuota(quota); - } -} - -void Pipe::pinSources(size_t executor_number) -{ - for (auto & processor : processors) - { - if (auto * source = dynamic_cast(processor.get())) - source->setStream(executor_number); - } -} - -void Pipe::enableQuota() -{ - for (auto & processor : processors) - { - if (auto * source = dynamic_cast(processor.get())) - source->enableQuota(); - } -} */ } diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index c6d42f8b269..5854f61554a 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -79,6 +79,11 @@ public: /// Unite several pipes together. They should have same header. static Pipe unitePipes(Pipes pipes); + /// Specify quotas and limits for every ISourceWithProgress. + void setLimits(const SourceWithProgress::LocalLimits & limits); + void setQuota(const std::shared_ptr & quota); + void enableQuota(); + /// Do not allow to change the table while the processors of pipe are alive. void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } /// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible. @@ -108,8 +113,13 @@ private: std::vector> interpreter_context; std::vector storage_holders; - /// Destroy pipes and get processors. - static Processors detachProcessors(Pipe pipe) { return std::move(pipe.processors); } + /// If is set, all newly created processors will be added to this too. + /// It is needed for debug. See QueryPipelineProcessorsCollector. + Processors * collected_processors = nullptr; + + static Pipe unitePipes(Pipes pipes, Processors * collected_processors); + + friend class QueryPipeline; }; /* diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 1c9ddb722f1..33d232da449 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -123,7 +123,7 @@ static OutputPort * uniteTotals(const std::vector & ports, const B return totals_port; } -void QueryPipeline::init(Pipes pipes) +void QueryPipeline::init(Pipe pipe) { if (initialized()) throw Exception("Pipeline has already been initialized.", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index f66d2a220cf..5020a2e8964 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -22,54 +22,6 @@ class QueryPipelineProcessorsCollector; class QueryPipeline { -private: - /// It's a wrapper over std::vector - /// Is needed to support invariant for max_parallel_streams (see comment below). - class Streams - { - public: - auto size() const { return data.size(); } - bool empty() const { return size() == 0; } - auto begin() { return data.begin(); } - auto end() { return data.end(); } - auto & front() { return data.front(); } - auto & back() { return data.back(); } - auto & at(size_t pos) { return data.at(pos); } - auto & operator[](size_t pos) { return data[pos]; } - - void clear() { data.clear(); } - void reserve(size_t size_) { data.reserve(size_); } - - void addStream(OutputPort * port, size_t port_max_parallel_streams) - { - data.push_back(port); - max_parallel_streams = std::max(max_parallel_streams + port_max_parallel_streams, data.size()); - } - - void addStreams(Streams & other) - { - data.insert(data.end(), other.begin(), other.end()); - max_parallel_streams = std::max(max_parallel_streams + other.max_parallel_streams, data.size()); - } - - void assign(std::initializer_list list) - { - data = list; - max_parallel_streams = std::max(max_parallel_streams, data.size()); - } - - size_t maxParallelStreams() const { return max_parallel_streams; } - - private: - std::vector data; - - /// It is the max number of processors which can be executed in parallel for each step. - /// Logically, it is the upper limit on the number of threads needed to execute this pipeline. - /// Initially, it is the number of sources. It may be increased after resize, aggregation, etc. - /// This number is never decreased, and it is calculated as max(streams.size()) over all streams while building. - size_t max_parallel_streams = 0; - }; - public: class ProcessorsContainer @@ -101,7 +53,7 @@ public: QueryPipeline & operator= (QueryPipeline && rhs); /// All pipes must have same header. - void init(Pipes pipes); + void init(Pipe pipe); bool initialized() { return !processors.empty(); } bool isCompleted() { return initialized() && streams.empty(); } diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.cpp b/src/Processors/QueryPlan/ReadFromStorageStep.cpp index efe8767d27d..39b0743fe38 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromStorageStep.cpp @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB { @@ -37,34 +36,44 @@ ReadFromStorageStep::ReadFromStorageStep( /// Note: we read from storage in constructor of step because we don't know real header before reading. /// It will be fixed when storage return QueryPlanStep itself. - Pipes pipes = storage->read(required_columns, metadata_snapshot, query_info, *context, processing_stage, max_block_size, max_streams); + Pipe pipe = storage->read(required_columns, metadata_snapshot, query_info, *context, processing_stage, max_block_size, max_streams); - if (pipes.empty()) + if (pipe.empty()) { - Pipe pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID()))); + pipe = Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID()))); if (query_info.prewhere_info) { if (query_info.prewhere_info->alias_actions) - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), query_info.prewhere_info->alias_actions)); + { + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, query_info.prewhere_info->alias_actions); + }); + } - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, query_info.prewhere_info->prewhere_actions, query_info.prewhere_info->prewhere_column_name, - query_info.prewhere_info->remove_prewhere_column)); + query_info.prewhere_info->remove_prewhere_column); + }); // To remove additional columns // In some cases, we did not read any marks so that the pipeline.streams is empty // Thus, some columns in prewhere are not removed as expected // This leads to mismatched header in distributed table if (query_info.prewhere_info->remove_columns_actions) - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), query_info.prewhere_info->remove_columns_actions)); + { + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, query_info.prewhere_info->remove_columns_actions); + }); + } } - - pipes.emplace_back(std::move(pipe)); } pipeline = std::make_unique(); @@ -104,20 +113,16 @@ ReadFromStorageStep::ReadFromStorageStep( auto quota = context->getQuota(); - for (auto & pipe : pipes) - { - if (!options.ignore_limits) - pipe.setLimits(limits); + if (!options.ignore_limits) + pipe.setLimits(limits); - if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete)) - pipe.setQuota(quota); - } + if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete)) + pipe.setQuota(quota); } - for (auto & pipe : pipes) - pipe.enableQuota(); + pipe.enableQuota(); - pipeline->init(std::move(pipes)); + pipeline->init(std::move(pipe)); pipeline->addInterpreterContext(std::move(context)); pipeline->addStorageHolder(std::move(storage)); From f428c2da707bab62bf18abbec831af89dc2ad4e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Jul 2020 02:09:47 +0300 Subject: [PATCH 007/374] Add a test --- src/Common/tests/CMakeLists.txt | 3 + src/Common/tests/average.cpp | 483 ++++++++++++++++++++++++++++++++ 2 files changed, 486 insertions(+) create mode 100644 src/Common/tests/average.cpp diff --git a/src/Common/tests/CMakeLists.txt b/src/Common/tests/CMakeLists.txt index 2653ab30c29..838150cd1bd 100644 --- a/src/Common/tests/CMakeLists.txt +++ b/src/Common/tests/CMakeLists.txt @@ -79,3 +79,6 @@ target_link_libraries (memory_statistics_os_perf PRIVATE clickhouse_common_io) add_executable (procfs_metrics_provider_perf procfs_metrics_provider_perf.cpp) target_link_libraries (procfs_metrics_provider_perf PRIVATE clickhouse_common_io) + +add_executable (average average.cpp) +target_link_libraries (average PRIVATE clickhouse_common_io) diff --git a/src/Common/tests/average.cpp b/src/Common/tests/average.cpp new file mode 100644 index 00000000000..9e73a09a860 --- /dev/null +++ b/src/Common/tests/average.cpp @@ -0,0 +1,483 @@ +#include +#include + +#include + +#include +#include +#include +#include +#include + + +/** This test program evaluates different solutions for a simple degenerate task: + * Aggregate data by UInt8 key, calculate "avg" function on Float values. + * + * It tests the overhead of various data structures in comparison to the minimal code doing the same task. + * It also tests what does it cost to access aggregation state via single pointer indirection. + * Also it evaluate various ways to unroll the loop. + * And finally it compares with one solution involving bucket sort. + * + * How to use: + * + * for i in {1..10}; do src/Common/tests/average 100000000 1; done + * + * You will find the numbers for various options below. + */ + + +using namespace DB; + +using Float = Float32; + +struct State +{ + Float sum = 0; + size_t count = 0; + + void add(Float value) + { + sum += value; + ++count; + } + + Float result() const + { + return sum / count; + } + + bool operator!() const + { + return !count; + } +}; + +using StatePtr = State *; + + +Float NO_INLINE baseline_baseline(const PODArray & keys, const PODArray & values) +{ + Arena arena; + HashMap map; + + size_t size = keys.size(); + for (size_t i = 0; i < size; ++i) + { + StatePtr & place = map[keys[i]]; + if (unlikely(!place)) + place = new (arena.alloc()) State(); + + place->add(values[i]); + } + + return map[0]->result(); +} + + +Float NO_INLINE baseline(const PODArray & keys, const PODArray & values) +{ + Arena arena; + FixedHashMap map; + + size_t size = keys.size(); + for (size_t i = 0; i < size; ++i) + { + StatePtr & place = map[keys[i]]; + if (unlikely(!place)) + place = new (arena.alloc()) State(); + + place->add(values[i]); + } + + return map[0]->result(); +} + + +template +using FixedImplicitZeroHashMap = FixedHashMap< + Key, + Mapped, + FixedHashMapImplicitZeroCell>; + +Float NO_INLINE implicit_zero(const PODArray & keys, const PODArray & values) +{ + Arena arena; + FixedImplicitZeroHashMap map; + + size_t size = keys.size(); + for (size_t i = 0; i < size; ++i) + { + StatePtr & place = map[keys[i]]; + if (unlikely(!place)) + place = new (arena.alloc()) State(); + + place->add(values[i]); + } + + return map[0]->result(); +} + + +template +using FixedHashMapWithCalculatedSize = FixedHashMap< + Key, + Mapped, + FixedHashMapCell, + FixedHashTableCalculatedSize>>; + +Float NO_INLINE calculated_size(const PODArray & keys, const PODArray & values) +{ + Arena arena; + FixedHashMapWithCalculatedSize map; + + size_t size = keys.size(); + for (size_t i = 0; i < size; ++i) + { + StatePtr & place = map[keys[i]]; + if (unlikely(!place)) + place = new (arena.alloc()) State(); + + place->add(values[i]); + } + + return map[0]->result(); +} + + +Float NO_INLINE implicit_zero_and_calculated_size(const PODArray & keys, const PODArray & values) +{ + Arena arena; + FixedImplicitZeroHashMapWithCalculatedSize map; + + size_t size = keys.size(); + for (size_t i = 0; i < size; ++i) + { + StatePtr & place = map[keys[i]]; + if (unlikely(!place)) + place = new (arena.alloc()) State(); + + place->add(values[i]); + } + + return map[0]->result(); +} + +Float NO_INLINE init_out_of_the_loop(const PODArray & keys, const PODArray & values) +{ + Arena arena; + FixedImplicitZeroHashMapWithCalculatedSize map; + + for (size_t i = 0; i < 256; ++i) + map[i] = new (arena.alloc()) State(); + + size_t size = keys.size(); + for (size_t i = 0; i < size; ++i) + { + StatePtr & place = map[keys[i]]; + place->add(values[i]); + } + + return map[0]->result(); +} + +Float NO_INLINE embedded_states(const PODArray & keys, const PODArray & values) +{ + FixedImplicitZeroHashMapWithCalculatedSize map; + + size_t size = keys.size(); + for (size_t i = 0; i < size; ++i) + { + State & place = map[keys[i]]; + place.add(values[i]); + } + + return map[0].result(); +} + +Float NO_INLINE simple_lookup_table(const PODArray & keys, const PODArray & values) +{ + Arena arena; + StatePtr map[256]{}; + + size_t size = keys.size(); + for (size_t i = 0; i < size; ++i) + { + StatePtr & place = map[keys[i]]; + if (unlikely(!place)) + place = new (arena.alloc()) State(); + + place->add(values[i]); + } + + return map[0]->result(); +} + +Float NO_INLINE simple_lookup_table_embedded_states(const PODArray & keys, const PODArray & values) +{ + State map[256]{}; + + size_t size = keys.size(); + for (size_t i = 0; i < size; ++i) + map[keys[i]].add(values[i]); + + return map[0].result(); +} + +template +Float NO_INLINE unrolled(const PODArray & keys, const PODArray & values) +{ + Arena arena; + FixedImplicitZeroHashMapWithCalculatedSize map; + + size_t size = keys.size(); + size_t i = 0; + + size_t size_unrolled = size / UNROLL_COUNT * UNROLL_COUNT; + for (; i < size_unrolled; i += UNROLL_COUNT) + { + StatePtr places[UNROLL_COUNT]; + for (size_t j = 0; j < UNROLL_COUNT; ++j) + { + StatePtr & place = map[keys[i + j]]; + if (unlikely(!place)) + place = new (arena.alloc()) State(); + + places[j] = place; + } + + for (size_t j = 0; j < UNROLL_COUNT; ++j) + places[j]->add(values[i + j]); + } + + for (; i < size; ++i) + { + StatePtr & place = map[keys[i]]; + if (unlikely(!place)) + place = new (arena.alloc()) State(); + + place->add(values[i]); + } + + return map[0]->result(); +} + +template +Float NO_INLINE simple_lookup_table_unrolled(const PODArray & keys, const PODArray & values) +{ + Arena arena; + StatePtr map[256]{}; + + size_t size = keys.size(); + size_t i = 0; + + size_t size_unrolled = size / UNROLL_COUNT * UNROLL_COUNT; + for (; i < size_unrolled; i += UNROLL_COUNT) + { + StatePtr places[UNROLL_COUNT]; + for (size_t j = 0; j < UNROLL_COUNT; ++j) + { + StatePtr & place = map[keys[i + j]]; + if (unlikely(!place)) + place = new (arena.alloc()) State(); + + places[j] = place; + } + + for (size_t j = 0; j < UNROLL_COUNT; ++j) + places[j]->add(values[i + j]); + } + + for (; i < size; ++i) + { + StatePtr & place = map[keys[i]]; + if (unlikely(!place)) + place = new (arena.alloc()) State(); + + place->add(values[i]); + } + + return map[0]->result(); +} + +template +Float NO_INLINE embedded_states_unrolled(const PODArray & keys, const PODArray & values) +{ + FixedImplicitZeroHashMapWithCalculatedSize map; + + size_t size = keys.size(); + size_t i = 0; + + size_t size_unrolled = size / UNROLL_COUNT * UNROLL_COUNT; + for (; i < size_unrolled; i += UNROLL_COUNT) + { + StatePtr places[UNROLL_COUNT]; + for (size_t j = 0; j < UNROLL_COUNT; ++j) + places[j] = &map[keys[i + j]]; + + for (size_t j = 0; j < UNROLL_COUNT; ++j) + places[j]->add(values[i + j]); + } + + for (; i < size; ++i) + { + State & place = map[keys[i]]; + place.add(values[i]); + } + + return map[0].result(); +} + +template +Float NO_INLINE simple_lookup_table_embedded_states_unrolled(const PODArray & keys, const PODArray & values) +{ + State map[256]{}; + + size_t size = keys.size(); + size_t i = 0; + + size_t size_unrolled = size / UNROLL_COUNT * UNROLL_COUNT; + for (; i < size_unrolled; i += UNROLL_COUNT) + { + StatePtr places[UNROLL_COUNT]; + for (size_t j = 0; j < UNROLL_COUNT; ++j) + places[j] = &map[keys[i + j]]; + + for (size_t j = 0; j < UNROLL_COUNT; ++j) + places[j]->add(values[i + j]); + } + + for (; i < size; ++i) + { + State & place = map[keys[i]]; + place.add(values[i]); + } + + return map[0].result(); +} + + +template +Float NO_INLINE microsort(const PODArray & keys, const PODArray & values) +{ + State map[256]{}; + + size_t size = keys.size(); + + /// Calculate histograms of keys. + + using CountType = UInt32; + + static constexpr size_t HISTOGRAM_SIZE = 256; + + CountType count[HISTOGRAM_SIZE * UNROLL_COUNT]{}; + size_t unrolled_size = size / UNROLL_COUNT * UNROLL_COUNT; + + for (const UInt8 * elem = keys.data(); elem < keys.data() + unrolled_size; elem += UNROLL_COUNT) + for (size_t i = 0; i < UNROLL_COUNT; ++i) + ++count[i * HISTOGRAM_SIZE + elem[i]]; + + for (const UInt8 * elem = keys.data() + unrolled_size; elem < keys.data() + size; ++elem) + ++count[*elem]; + + for (size_t i = 0; i < HISTOGRAM_SIZE; ++i) + for (size_t j = 1; j < UNROLL_COUNT; ++j) + count[i] += count[j * HISTOGRAM_SIZE + i]; + + /// Row indices in a batch for each key. + + PODArray indices(size); + UInt32 * positions[HISTOGRAM_SIZE]; + positions[0] = indices.data(); + + for (size_t i = 1; i < HISTOGRAM_SIZE; ++i) + positions[i] = positions[i - 1] + count[i - 1]; + + for (size_t i = 0; i < size; ++i) + *positions[keys[i]]++ = i; + + /// Update states. + + UInt32 * idx = indices.data(); + for (size_t i = 0; i < HISTOGRAM_SIZE; ++i) + for (; idx < positions[i]; ++idx) + map[i].add(values[*idx]); + + return map[0].result(); +} + + +int main(int argc, char ** argv) +{ + size_t size = argc > 1 ? std::stoull(argv[1]) : 1000000000; + size_t variant = argc > 2 ? std::stoull(argv[2]) : 1; + + PODArray keys(size); + PODArray values(size); + + /// Fill source data + for (size_t i = 0; i < size; ++i) + { + keys[i] = __builtin_ctz(i + 1); /// Make keys to have just slightly more realistic distribution. + values[i] = i; /// The distribution of values does not affect execution speed. + } + + /// Aggregate + Stopwatch watch; + Float res; + + switch (variant) + { + case 0: res = baseline(keys, values); break; + case 1: res = implicit_zero(keys, values); break; + case 2: res = calculated_size(keys, values); break; + case 3: res = implicit_zero_and_calculated_size(keys, values); break; + case 4: res = init_out_of_the_loop(keys, values); break; + case 5: res = embedded_states(keys, values); break; + case 6: res = simple_lookup_table(keys, values); break; + case 7: res = simple_lookup_table_embedded_states(keys, values); break; + case 8: res = microsort<1>(keys, values); break; + case 9: res = baseline_baseline(keys, values); break; + + case 32: res = unrolled<2>(keys, values); break; + case 34: res = unrolled<4>(keys, values); break; + case 36: res = unrolled<6>(keys, values); break; + case 38: res = unrolled<8>(keys, values); break; + case 316: res = unrolled<16>(keys, values); break; + + case 52: res = embedded_states_unrolled<2>(keys, values); break; + case 54: res = embedded_states_unrolled<4>(keys, values); break; + case 56: res = embedded_states_unrolled<6>(keys, values); break; + case 58: res = embedded_states_unrolled<8>(keys, values); break; + case 516: res = embedded_states_unrolled<16>(keys, values); break; + + case 62: res = simple_lookup_table_unrolled<2>(keys, values); break; + case 64: res = simple_lookup_table_unrolled<4>(keys, values); break; + case 66: res = simple_lookup_table_unrolled<6>(keys, values); break; + case 68: res = simple_lookup_table_unrolled<8>(keys, values); break; + case 616: res = simple_lookup_table_unrolled<16>(keys, values); break; + + case 72: res = simple_lookup_table_embedded_states_unrolled<2>(keys, values); break; + case 74: res = simple_lookup_table_embedded_states_unrolled<4>(keys, values); break; + case 76: res = simple_lookup_table_embedded_states_unrolled<6>(keys, values); break; + case 78: res = simple_lookup_table_embedded_states_unrolled<8>(keys, values); break; + case 716: res = simple_lookup_table_embedded_states_unrolled<16>(keys, values); break; + + case 82: res = microsort<2>(keys, values); break; + case 84: res = microsort<4>(keys, values); break; + case 86: res = microsort<6>(keys, values); break; + case 88: res = microsort<8>(keys, values); break; + case 816: res = microsort<16>(keys, values); break; + + default: break; + } + + watch.stop(); + fmt::print("Aggregated (res = {}) in {} sec., {} million rows/sec., {} MiB/sec.\n", + res, + watch.elapsedSeconds(), + size_t(size / watch.elapsedSeconds() / 1000000), + size_t(size * (sizeof(Float) + sizeof(UInt8)) / watch.elapsedSeconds() / 1000000)); + + return 0; +} + From ba6d5d00d97ba11ce4a61a2c98cf8b75131d8aa8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Jul 2020 02:10:11 +0300 Subject: [PATCH 008/374] Apply one of the variants --- src/AggregateFunctions/IAggregateFunction.h | 57 ++++++++++++++------- 1 file changed, 39 insertions(+), 18 deletions(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index a0876d457b8..251cf521d1a 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -222,24 +222,6 @@ public: static_cast(this)->add(places[i] + place_offset, columns, i, arena); } - void addBatchLookupTable8( - size_t batch_size, - AggregateDataPtr * places, - size_t place_offset, - std::function init, - const UInt8 * key, - const IColumn ** columns, - Arena * arena) const override - { - for (size_t i = 0; i < batch_size; ++i) - { - AggregateDataPtr & place = places[key[i]]; - if (unlikely(!place)) - init(place); - static_cast(this)->add(place + place_offset, columns, i, arena); - } - } - void addBatchSinglePlace(size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override { for (size_t i = 0; i < batch_size; ++i) @@ -274,6 +256,45 @@ public: current_offset = next_offset; } } + + void addBatchLookupTable8( + size_t batch_size, + AggregateDataPtr * map, + size_t place_offset, + std::function init, + const UInt8 * key, + const IColumn ** columns, + Arena * arena) const override + { + static constexpr size_t UNROLL_COUNT = 8; + + size_t i = 0; + + size_t batch_size_unrolled = batch_size / UNROLL_COUNT * UNROLL_COUNT; + for (; i < batch_size_unrolled; i += UNROLL_COUNT) + { + AggregateDataPtr places[UNROLL_COUNT]; + for (size_t j = 0; j < UNROLL_COUNT; ++j) + { + AggregateDataPtr & place = map[key[i + j]]; + if (unlikely(!place)) + init(place); + + places[j] = place; + } + + for (size_t j = 0; j < UNROLL_COUNT; ++j) + static_cast(this)->add(places[j] + place_offset, columns, i + j, arena); + } + + for (; i < batch_size; ++i) + { + AggregateDataPtr & place = map[key[i]]; + if (unlikely(!place)) + init(place); + static_cast(this)->add(place + place_offset, columns, i, arena); + } + } }; From 4d5ba3571bc005fb9d17070e6abcedc02215bbe8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 30 Jul 2020 02:14:55 +0300 Subject: [PATCH 009/374] Update average.cpp --- src/Common/tests/average.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/tests/average.cpp b/src/Common/tests/average.cpp index 9e73a09a860..3e793245e26 100644 --- a/src/Common/tests/average.cpp +++ b/src/Common/tests/average.cpp @@ -15,7 +15,7 @@ * * It tests the overhead of various data structures in comparison to the minimal code doing the same task. * It also tests what does it cost to access aggregation state via single pointer indirection. - * Also it evaluate various ways to unroll the loop. + * Also it evaluates various ways to unroll the loop. * And finally it compares with one solution involving bucket sort. * * How to use: From d94717c699c70a20774b9664708585ac2c6afb2c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Jul 2020 03:21:22 +0300 Subject: [PATCH 010/374] Add one more method to the test --- src/Common/tests/average.cpp | 69 ++++++++++++++++++++++++++++++++++++ 1 file changed, 69 insertions(+) diff --git a/src/Common/tests/average.cpp b/src/Common/tests/average.cpp index 3e793245e26..9ddb94eaf91 100644 --- a/src/Common/tests/average.cpp +++ b/src/Common/tests/average.cpp @@ -41,6 +41,36 @@ struct State ++count; } + template + void addBatch(const Float * ptr, size_t size) + { + /// Compiler cannot unroll this loop, do it manually. + /// (at least for floats, most likely due to the lack of -fassociative-math) + + Float partial_sums[unroll_count]{}; + + const auto * end = ptr + size; + const auto * unrolled_end = ptr + (size / unroll_count * unroll_count); + + while (ptr < unrolled_end) + { + for (size_t i = 0; i < unroll_count; ++i) + partial_sums[i] += ptr[i]; + ptr += unroll_count; + } + + for (size_t i = 0; i < unroll_count; ++i) + sum += partial_sums[i]; + + while (ptr < end) + { + sum += *ptr; + ++ptr; + } + + count += size; + } + Float result() const { return sum / count; @@ -406,6 +436,44 @@ Float NO_INLINE microsort(const PODArray & keys, const PODArray & } +Float NO_INLINE buffered(const PODArray & keys, const PODArray & values) +{ + State map[256]{}; + + static constexpr size_t BUF_SIZE = 16384 / 256 / sizeof(Float); /// Should fit in L1d. + + Float buffers[256 * BUF_SIZE]; + Float * ptrs[256]; + + for (size_t i = 0; i < 256; ++i) + ptrs[i] = &buffers[i * BUF_SIZE]; + + size_t size = keys.size(); + const auto * key = keys.data(); + const auto * key_end = key + size; + const auto * value = values.data(); + + while (key < key_end) + { + *ptrs[*key] = *value; + + if (++ptrs[*key] == &buffers[(*key + 1) * BUF_SIZE]) /// Calculation is better than L1d load of cached end pointer. + { + ptrs[*key] -= BUF_SIZE; + map[*key].addBatch(ptrs[*key], BUF_SIZE); + } + + ++key; + ++value; + } + + for (size_t i = 0; i < 256; ++i) + map[i].addBatch<4>(&buffers[i * BUF_SIZE], ptrs[i] - &buffers[i * BUF_SIZE]); + + return map[0].result(); +} + + int main(int argc, char ** argv) { size_t size = argc > 1 ? std::stoull(argv[1]) : 1000000000; @@ -437,6 +505,7 @@ int main(int argc, char ** argv) case 7: res = simple_lookup_table_embedded_states(keys, values); break; case 8: res = microsort<1>(keys, values); break; case 9: res = baseline_baseline(keys, values); break; + case 10: res = buffered(keys, values); break; case 32: res = unrolled<2>(keys, values); break; case 34: res = unrolled<4>(keys, values); break; From a7e5feb8b40b72e5dc2ce5c24578452d6f6e73c6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Jul 2020 03:58:19 +0300 Subject: [PATCH 011/374] Added another method --- src/Common/tests/average.cpp | 39 +++++++++++++++++++++++++++++++++++- 1 file changed, 38 insertions(+), 1 deletion(-) diff --git a/src/Common/tests/average.cpp b/src/Common/tests/average.cpp index 9ddb94eaf91..abaa6644039 100644 --- a/src/Common/tests/average.cpp +++ b/src/Common/tests/average.cpp @@ -71,6 +71,12 @@ struct State count += size; } + void merge(const State & other) + { + sum += other.sum; + count += other.count; + } + Float result() const { return sum / count; @@ -474,6 +480,30 @@ Float NO_INLINE buffered(const PODArray & keys, const PODArray & v } +template +Float NO_INLINE really_unrolled(const PODArray & keys, const PODArray & values) +{ + State map[256 * UNROLL_COUNT]{}; + + size_t size = keys.size(); + size_t i = 0; + + size_t size_unrolled = size / UNROLL_COUNT * UNROLL_COUNT; + for (; i < size_unrolled; i += UNROLL_COUNT) + for (size_t j = 0; j < UNROLL_COUNT; ++j) + map[256 * j + keys[i + j]].add(values[i + j]); + + for (size_t key = 0; key < 256; ++key) + for (size_t j = 1; j < UNROLL_COUNT; ++j) + map[key].merge(map[256 * j + key]); + + for (; i < size; ++i) + map[keys[i]].add(values[i]); + + return map[0].result(); +} + + int main(int argc, char ** argv) { size_t size = argc > 1 ? std::stoull(argv[1]) : 1000000000; @@ -486,7 +516,7 @@ int main(int argc, char ** argv) for (size_t i = 0; i < size; ++i) { keys[i] = __builtin_ctz(i + 1); /// Make keys to have just slightly more realistic distribution. - values[i] = i; /// The distribution of values does not affect execution speed. + values[i] = 1234.5; /// The distribution of values does not affect execution speed. } /// Aggregate @@ -506,6 +536,7 @@ int main(int argc, char ** argv) case 8: res = microsort<1>(keys, values); break; case 9: res = baseline_baseline(keys, values); break; case 10: res = buffered(keys, values); break; + case 11: res = really_unrolled<1>(keys, values); break; case 32: res = unrolled<2>(keys, values); break; case 34: res = unrolled<4>(keys, values); break; @@ -537,6 +568,12 @@ int main(int argc, char ** argv) case 88: res = microsort<8>(keys, values); break; case 816: res = microsort<16>(keys, values); break; + case 112: res = really_unrolled<2>(keys, values); break; + case 114: res = really_unrolled<4>(keys, values); break; + case 116: res = really_unrolled<5>(keys, values); break; + case 118: res = really_unrolled<8>(keys, values); break; + case 1116: res = really_unrolled<16>(keys, values); break; + default: break; } From a9998746ea36f93e8b909a45bebffe7ebcf2ba19 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Jul 2020 04:27:49 +0300 Subject: [PATCH 012/374] Added fully unrolled method --- src/AggregateFunctions/IAggregateFunction.h | 83 +++++++++++++++++++++ 1 file changed, 83 insertions(+) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 251cf521d1a..e5109031233 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -337,6 +337,89 @@ public: { return alignof(Data); } + + void addBatchLookupTable8( + size_t batch_size, + AggregateDataPtr * map, + size_t place_offset, + std::function init, + const UInt8 * key, + const IColumn ** columns, + Arena * arena) const override + { + const Derived & func = *static_cast(this); + + if (func.allocatesMemoryInArena() || sizeof(Data) > 16) + { + IAggregateFunctionHelper::addBatchLookupTable8(batch_size, map, place_offset, init, key, columns, arena); + return; + } + + static constexpr size_t UNROLL_COUNT = 8; + + Data places[256 * UNROLL_COUNT]; + bool has_data[256 * UNROLL_COUNT]{}; + + size_t i = 0; + + size_t batch_size_unrolled = batch_size / UNROLL_COUNT * UNROLL_COUNT; + for (; i < batch_size_unrolled; i += UNROLL_COUNT) + { + for (size_t j = 0; j < UNROLL_COUNT; ++j) + { + size_t idx = j * 256 + key[i + j]; + if (unlikely(!has_data[idx])) + { + new (&places[idx]) Data; + has_data[idx] = true; + } + func.add(reinterpret_cast(&places[idx]), columns, i + j, nullptr); + } + } + + for (size_t k = 0; k < 256; ++k) + { + for (size_t j = 1; j < UNROLL_COUNT; ++j) + { + if (has_data[j * 256 + k]) + { + if (unlikely(!has_data[k])) + { + new (&places[k]) Data; + has_data[k] = true; + } + + func.merge( + reinterpret_cast(&places[k]), + reinterpret_cast(&places[256 * j + k]), + nullptr); + } + } + } + + for (; i < batch_size; ++i) + { + size_t idx = key[i]; + if (unlikely(!has_data[idx])) + { + new (&places[idx]) Data; + has_data[idx] = true; + } + func.add(reinterpret_cast(&places[idx]), columns, i, nullptr); + } + + for (size_t k = 0; k < 256; ++k) + { + if (has_data[k]) + { + AggregateDataPtr & place = map[k]; + if (unlikely(!place)) + init(place); + + func.merge(place + place_offset, reinterpret_cast(&places[k]), arena); + } + } + } }; From f29c0af9aa3ba5bff04983ea2a91e450c575d190 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Jul 2020 04:35:50 +0300 Subject: [PATCH 013/374] Simpler code --- src/AggregateFunctions/IAggregateFunction.h | 50 +++++++++------------ 1 file changed, 21 insertions(+), 29 deletions(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index e5109031233..22c55ee0521 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -349,19 +349,25 @@ public: { const Derived & func = *static_cast(this); + /// If the function is complex or too large, use more generic algorithm. + if (func.allocatesMemoryInArena() || sizeof(Data) > 16) { IAggregateFunctionHelper::addBatchLookupTable8(batch_size, map, place_offset, init, key, columns, arena); return; } + /// Will use UNROLL_COUNT number of lookup tables. + static constexpr size_t UNROLL_COUNT = 8; Data places[256 * UNROLL_COUNT]; - bool has_data[256 * UNROLL_COUNT]{}; + bool has_data[256 * UNROLL_COUNT]{}; /// Separate flags array to avoid heavy initialization. size_t i = 0; + /// Aggregate data into different lookup tables. + size_t batch_size_unrolled = batch_size / UNROLL_COUNT * UNROLL_COUNT; for (; i < batch_size_unrolled; i += UNROLL_COUNT) { @@ -377,47 +383,33 @@ public: } } + /// Merge data from every lookup table to the final destination. + for (size_t k = 0; k < 256; ++k) { - for (size_t j = 1; j < UNROLL_COUNT; ++j) + for (size_t j = 0; j < UNROLL_COUNT; ++j) { if (has_data[j * 256 + k]) { - if (unlikely(!has_data[k])) - { - new (&places[k]) Data; - has_data[k] = true; - } + AggregateDataPtr & place = map[k]; + if (unlikely(!place)) + init(place); - func.merge( - reinterpret_cast(&places[k]), - reinterpret_cast(&places[256 * j + k]), - nullptr); + func.merge(place + place_offset, reinterpret_cast(&places[256 * j + k]), arena); } } } + /// Process tails and add directly to the final destination. + for (; i < batch_size; ++i) { - size_t idx = key[i]; - if (unlikely(!has_data[idx])) - { - new (&places[idx]) Data; - has_data[idx] = true; - } - func.add(reinterpret_cast(&places[idx]), columns, i, nullptr); - } + size_t k = key[i]; + AggregateDataPtr & place = map[k]; + if (unlikely(!place)) + init(place); - for (size_t k = 0; k < 256; ++k) - { - if (has_data[k]) - { - AggregateDataPtr & place = map[k]; - if (unlikely(!place)) - init(place); - - func.merge(place + place_offset, reinterpret_cast(&places[k]), arena); - } + func.add(place + place_offset, columns, i, nullptr); } } }; From eb534ebee37ca6f7065fe8bb6400e38eb4600ead Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Jul 2020 04:51:57 +0300 Subject: [PATCH 014/374] Fix warning --- src/AggregateFunctions/IAggregateFunction.h | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 22c55ee0521..4498d276f01 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -357,6 +357,12 @@ public: return; } + /// The warning about array bounds is irrelevant if the function not allocatesMemoryInArena. +#if !__clang__ +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Warray-bounds" +#endif + /// Will use UNROLL_COUNT number of lookup tables. static constexpr size_t UNROLL_COUNT = 8; @@ -411,6 +417,10 @@ public: func.add(place + place_offset, columns, i, nullptr); } + +#if !__clang__ +#pragma GCC diagnostic pop +#endif } }; From 373cd918fbcc076808c354a0e4eddf1750fd5662 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Jul 2020 05:24:04 +0300 Subject: [PATCH 015/374] Remove warning suppression --- src/AggregateFunctions/IAggregateFunction.h | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 4498d276f01..22c55ee0521 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -357,12 +357,6 @@ public: return; } - /// The warning about array bounds is irrelevant if the function not allocatesMemoryInArena. -#if !__clang__ -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Warray-bounds" -#endif - /// Will use UNROLL_COUNT number of lookup tables. static constexpr size_t UNROLL_COUNT = 8; @@ -417,10 +411,6 @@ public: func.add(place + place_offset, columns, i, nullptr); } - -#if !__clang__ -#pragma GCC diagnostic pop -#endif } }; From e241df55415d0d5d32d051dd2392fce99b00577d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Jul 2020 05:28:10 +0300 Subject: [PATCH 016/374] Add warning suppression for gcc --- src/AggregateFunctions/AggregateFunctionUniqUpTo.h | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionUniqUpTo.h b/src/AggregateFunctions/AggregateFunctionUniqUpTo.h index 2a48e0fb182..5194c0de4c8 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqUpTo.h +++ b/src/AggregateFunctions/AggregateFunctionUniqUpTo.h @@ -17,6 +17,11 @@ #include +#if !__clang__ +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Warray-bounds" +#endif + namespace DB { @@ -250,3 +255,8 @@ public: } + +#if !__clang__ +#pragma GCC diagnostic pop +#endif + From 05bfb70c7acd42cef5e957886ee47ffa91748de7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 31 Jul 2020 00:59:00 +0300 Subject: [PATCH 017/374] Addition to Revert "Less indirection in FixedHashTable" --- src/Common/tests/average.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Common/tests/average.cpp b/src/Common/tests/average.cpp index abaa6644039..e1a617524b3 100644 --- a/src/Common/tests/average.cpp +++ b/src/Common/tests/average.cpp @@ -129,12 +129,6 @@ Float NO_INLINE baseline(const PODArray & keys, const PODArray & v } -template -using FixedImplicitZeroHashMap = FixedHashMap< - Key, - Mapped, - FixedHashMapImplicitZeroCell>; - Float NO_INLINE implicit_zero(const PODArray & keys, const PODArray & values) { Arena arena; From 6d5b9c36e007303b6627393e8cf0fe3bbab6e3ce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 31 Jul 2020 01:00:01 +0300 Subject: [PATCH 018/374] Lower unroll count --- src/AggregateFunctions/IAggregateFunction.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 22c55ee0521..c3968db189f 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -359,7 +359,7 @@ public: /// Will use UNROLL_COUNT number of lookup tables. - static constexpr size_t UNROLL_COUNT = 8; + static constexpr size_t UNROLL_COUNT = 4; Data places[256 * UNROLL_COUNT]; bool has_data[256 * UNROLL_COUNT]{}; /// Separate flags array to avoid heavy initialization. From c58d0b428d736b57fc32c363e18a41fee9acea3c Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 13 Apr 2020 17:32:02 +0800 Subject: [PATCH 019/374] add myrepl-client.cpp --- programs/CMakeLists.txt | 1 + programs/myrepl-client/CMakeLists.txt | 4 ++++ programs/myrepl-client/myrepl_client.cpp | 9 +++++++++ 3 files changed, 14 insertions(+) create mode 100644 programs/myrepl-client/CMakeLists.txt create mode 100644 programs/myrepl-client/myrepl_client.cpp diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index b36a2ff8194..62d5d523fc4 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -84,6 +84,7 @@ add_subdirectory (compressor) add_subdirectory (copier) add_subdirectory (format) add_subdirectory (obfuscator) +add_subdirectory (myrepl-client) if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) add_subdirectory (odbc-bridge) diff --git a/programs/myrepl-client/CMakeLists.txt b/programs/myrepl-client/CMakeLists.txt new file mode 100644 index 00000000000..b3d22409f1e --- /dev/null +++ b/programs/myrepl-client/CMakeLists.txt @@ -0,0 +1,4 @@ +include_directories(${CMAKE_CURRENT_BINARY_DIR}) + +add_executable(myrepl-client myrepl_client.cpp) +target_link_libraries(myrepl-client) \ No newline at end of file diff --git a/programs/myrepl-client/myrepl_client.cpp b/programs/myrepl-client/myrepl_client.cpp new file mode 100644 index 00000000000..6c3e74b0adf --- /dev/null +++ b/programs/myrepl-client/myrepl_client.cpp @@ -0,0 +1,9 @@ +#include + + +int main(int argc, char ** argv) +{ + std::cout << "Try: " << argv[1] << std::endl; + + return 0; +} \ No newline at end of file From 63c0f495b95333854736c6b28d3fd0f437065252 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Thu, 16 Apr 2020 21:27:06 +0800 Subject: [PATCH 020/374] MySQL: make MySQLProtocol work in server&client duplex mode --- programs/CMakeLists.txt | 1 - programs/myrepl-client/CMakeLists.txt | 4 - programs/myrepl-client/myrepl_client.cpp | 9 - src/Core/MySQLClient.cpp | 81 +++++++++ src/Core/MySQLClient.h | 50 ++++++ src/Core/MySQLProtocol.h | 210 ++++++++++++++++++++--- src/Core/tests/CMakeLists.txt | 3 + src/Core/tests/mysql_client.cpp | 13 ++ 8 files changed, 331 insertions(+), 40 deletions(-) delete mode 100644 programs/myrepl-client/CMakeLists.txt delete mode 100644 programs/myrepl-client/myrepl_client.cpp create mode 100644 src/Core/MySQLClient.cpp create mode 100644 src/Core/MySQLClient.h create mode 100644 src/Core/tests/mysql_client.cpp diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 62d5d523fc4..b36a2ff8194 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -84,7 +84,6 @@ add_subdirectory (compressor) add_subdirectory (copier) add_subdirectory (format) add_subdirectory (obfuscator) -add_subdirectory (myrepl-client) if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) add_subdirectory (odbc-bridge) diff --git a/programs/myrepl-client/CMakeLists.txt b/programs/myrepl-client/CMakeLists.txt deleted file mode 100644 index b3d22409f1e..00000000000 --- a/programs/myrepl-client/CMakeLists.txt +++ /dev/null @@ -1,4 +0,0 @@ -include_directories(${CMAKE_CURRENT_BINARY_DIR}) - -add_executable(myrepl-client myrepl_client.cpp) -target_link_libraries(myrepl-client) \ No newline at end of file diff --git a/programs/myrepl-client/myrepl_client.cpp b/programs/myrepl-client/myrepl_client.cpp deleted file mode 100644 index 6c3e74b0adf..00000000000 --- a/programs/myrepl-client/myrepl_client.cpp +++ /dev/null @@ -1,9 +0,0 @@ -#include - - -int main(int argc, char ** argv) -{ - std::cout << "Try: " << argv[1] << std::endl; - - return 0; -} \ No newline at end of file diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp new file mode 100644 index 00000000000..882c7e83fd0 --- /dev/null +++ b/src/Core/MySQLClient.cpp @@ -0,0 +1,81 @@ +#include + +namespace DB +{ +using namespace MySQLProtocol; +using namespace MySQLProtocol::Authentication; + +namespace ErrorCodes +{ + extern const int NETWORK_ERROR; + extern const int SOCKET_TIMEOUT; + extern const int UNKNOWN_PACKET_FROM_SERVER; +} + +MySQLClient::MySQLClient(const String & _host, UInt16 _port, const String & _user, const String & _password, const String & _database) + : host(_host), port(_port), user(_user), password(_password), database(_database) +{ + client_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_PLUGIN_AUTH | CLIENT_SECURE_CONNECTION; +} + +void MySQLClient::connect() +{ + try + { + if (connected) + { + close(); + } + + socket = std::make_unique(); + address = DNSResolver::instance().resolveAddress(host, port); + socket->connect(*address); + + in = std::make_shared(*socket); + out = std::make_shared(*socket); + packet_sender = std::make_shared(*in, *out, seq); + connected = true; + + handshake(*in); + } + catch (Poco::Net::NetException & e) + { + close(); + throw NetException(e.displayText(), ErrorCodes::NETWORK_ERROR); + } + catch (Poco::TimeoutException & e) + { + close(); + throw NetException(e.displayText(), ErrorCodes::SOCKET_TIMEOUT); + } +} + +void MySQLClient::close() +{ + in = nullptr; + out = nullptr; + if (socket) + socket->close(); + socket = nullptr; + connected = false; +} + +/// https://dev.mysql.com/doc/internals/en/connection-phase-packets.html +void MySQLClient::handshake(ReadBuffer & payload) +{ + Handshake handshake; + handshake.readPayloadImpl(payload); + if (handshake.auth_plugin_name != mysql_native_password) + { + throw Exception( + "Only support " + mysql_native_password + " auth plugin name, but got " + handshake.auth_plugin_name, + ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); + } + + Native41 native41(password, handshake.auth_plugin_data); + String response = native41.getAuthPluginData(); + + HandshakeResponse handshakeResponse(client_capability_flags, 0, charset_utf8, user, database, handshake.auth_plugin_data, mysql_native_password); + packet_sender->sendPacket(handshakeResponse, true); +} +} diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h new file mode 100644 index 00000000000..921626d05c7 --- /dev/null +++ b/src/Core/MySQLClient.h @@ -0,0 +1,50 @@ +#pragma once +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +class MySQLClient +{ +public: + MySQLClient(const String & _host, UInt16 _port, const String & _user, const String & _password, const String & _database); + + void connect(); + + void close(); + +private: + String host; + UInt16 port; + String user; + String password; + String database; + + bool connected = false; + UInt32 client_capability_flags = 0; + + uint8_t seq = 0; + UInt8 charset_utf8 = 33; + String mysql_native_password = "mysql_native_password"; + + std::shared_ptr in; + std::shared_ptr out; + std::unique_ptr socket; + std::optional address; + + void handshake(ReadBuffer & payload); + +protected: + std::shared_ptr packet_sender; +}; +} diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index ee53276ee1b..55bdf987dee 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -227,17 +227,17 @@ protected: }; -class ClientPacket +class ReadPacket { public: - ClientPacket() = default; + ReadPacket() = default; - ClientPacket(ClientPacket &&) = default; + ReadPacket(ReadPacket &&) = default; - virtual void read(ReadBuffer & in, uint8_t & sequence_id) + virtual void readPayload(ReadBuffer & in, uint8_t & sequence_id) { PacketPayloadReadBuffer payload(in, sequence_id); - readPayload(payload); + readPayloadImpl(payload); if (!payload.eof()) { std::stringstream tmp; @@ -246,19 +246,19 @@ public: } } - virtual void readPayload(ReadBuffer & buf) = 0; + virtual void readPayloadImpl(ReadBuffer & buf) = 0; - virtual ~ClientPacket() = default; + virtual ~ReadPacket() = default; }; -class LimitedClientPacket : public ClientPacket +class LimitedReadPacket : public ReadPacket { public: - void read(ReadBuffer & in, uint8_t & sequence_id) override + void readPayload(ReadBuffer & in, uint8_t & sequence_id) override { LimitReadBuffer limited(in, 10000, true, "too long MySQL packet."); - ClientPacket::read(limited, sequence_id); + ReadPacket::readPayload(limited, sequence_id); } }; @@ -359,7 +359,6 @@ protected: virtual void writePayloadImpl(WriteBuffer & buffer) const = 0; }; - /* Writes and reads packets, keeping sequence-id. * Throws ProtocolError, if packet with incorrect sequence-id was received. */ @@ -387,9 +386,9 @@ public: { } - void receivePacket(ClientPacket & packet) + void receivePacket(ReadPacket & packet) { - packet.read(*in, sequence_id); + packet.readPayload(*in, sequence_id); } template @@ -435,8 +434,9 @@ size_t getLengthEncodedNumberSize(uint64_t x); size_t getLengthEncodedStringSize(const String & s); -class Handshake : public WritePacket +class Handshake : public WritePacket, ReadPacket { +public: int protocol_version = 0xa; String server_version; uint32_t connection_id; @@ -445,8 +445,10 @@ class Handshake : public WritePacket uint32_t status_flags; String auth_plugin_name; String auth_plugin_data; -public: - explicit Handshake(uint32_t capability_flags_, uint32_t connection_id_, String server_version_, String auth_plugin_name_, String auth_plugin_data_) + + Handshake() = default; + + Handshake(uint32_t capability_flags_, uint32_t connection_id_, String server_version_, String auth_plugin_name_, String auth_plugin_data_) : protocol_version(0xa) , server_version(std::move(server_version_)) , connection_id(connection_id_) @@ -458,7 +460,6 @@ public: { } -protected: size_t getPayloadSize() const override { return 26 + server_version.size() + auth_plugin_data.size() + auth_plugin_name.size(); @@ -480,16 +481,76 @@ protected: writeString(auth_plugin_name, buffer); writeChar(0x0, 1, buffer); } + + void readPayloadImpl(ReadBuffer & buffer) override + { + buffer.ignore(4); + + /// 1-byte: [0a] protocol version + buffer.readStrict(reinterpret_cast(&protocol_version), 1); + + /// string[NUL]: server version + readNullTerminated(server_version, buffer); + + /// 4-bytes: connection id + buffer.readStrict(reinterpret_cast(&connection_id), 4); + + /// 8-bytes: auth-plugin-data-part-1 + buffer.readStrict(reinterpret_cast(&auth_plugin_data), AUTH_PLUGIN_DATA_PART_1_LENGTH); + + /// 1-byte: [00] filler + buffer.ignore(1); + + /// 2-bytes: capability flags lower 2-bytes + buffer.readStrict(reinterpret_cast(&capability_flags), 2); + + /// 1-byte: character set + buffer.readStrict(reinterpret_cast(&character_set), 1); + + /// 2-bytes: status flags(ignored) + buffer.readStrict(reinterpret_cast(&status_flags), 2); + + /// 2-bytes: capability flags upper 2-bytes + buffer.readStrict((reinterpret_cast(&capability_flags)) + 2, 2); + + UInt8 auth_plugin_data_length = 0; + if (capability_flags & MySQLProtocol::CLIENT_PLUGIN_AUTH) + { + /// 1-byte: length of auth-plugin-data + buffer.readStrict(reinterpret_cast(&auth_plugin_data_length), 1); + } + else + { + /// 1-byte: [00] + buffer.ignore(1); + } + + /// string[10] reserved (all [00]) + buffer.ignore(10); + + if (capability_flags & MySQLProtocol::CLIENT_SECURE_CONNECTION) + { + UInt8 part2_length = (auth_plugin_data_length - AUTH_PLUGIN_DATA_PART_1_LENGTH) > 13 + ? 13 + : (auth_plugin_data_length - AUTH_PLUGIN_DATA_PART_1_LENGTH); + buffer.readStrict((reinterpret_cast(&auth_plugin_data)) + AUTH_PLUGIN_DATA_PART_1_LENGTH, part2_length); + } + + if (capability_flags & MySQLProtocol::CLIENT_PLUGIN_AUTH) + { + readNullTerminated(auth_plugin_name, buffer); + } + } }; -class SSLRequest : public ClientPacket +class SSLRequest : public ReadPacket { public: uint32_t capability_flags; uint32_t max_packet_size; uint8_t character_set; - void readPayload(ReadBuffer & buf) override + void readPayloadImpl(ReadBuffer & buf) override { buf.readStrict(reinterpret_cast(&capability_flags), 4); buf.readStrict(reinterpret_cast(&max_packet_size), 4); @@ -497,20 +558,94 @@ public: } }; -class HandshakeResponse : public LimitedClientPacket +class HandshakeResponse : public WritePacket, ReadPacket { public: uint32_t capability_flags = 0; uint32_t max_packet_size = 0; uint8_t character_set = 0; String username; - String auth_response; String database; + String auth_response; String auth_plugin_name; HandshakeResponse() = default; - void readPayload(ReadBuffer & payload) override + HandshakeResponse( + UInt32 _capability_flags, + UInt32 _max_packet_size, + UInt8 _character_set, + const String & _username, + const String & _database, + const String & _auth_response, + const String & _auth_plugin_name) + : capability_flags(_capability_flags) + , max_packet_size(_max_packet_size) + , character_set(_character_set) + , username(_username) + , database(_database) + , auth_response(_auth_response) + , auth_plugin_name(_auth_plugin_name){}; + + size_t getPayloadSize() const override + { + size_t size = 0; + size += 4 + 4 + 1; + if (capability_flags & CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA) + { + size += getLengthEncodedStringSize(auth_response); + } + else if (capability_flags & CLIENT_SECURE_CONNECTION) + { + size += (1 + auth_response.size()); + } + else + { + size += (auth_response.size() + 1); + } + if (capability_flags & CLIENT_CONNECT_WITH_DB) + { + size += (database.size() + 1); + } + if (capability_flags & CLIENT_PLUGIN_AUTH) + { + size += (auth_plugin_name.size() + 1); + } + return size; + } + + void writePayloadImpl(WriteBuffer & buffer) const override + { + buffer.write(reinterpret_cast(&capability_flags), 4); + buffer.write(reinterpret_cast(&max_packet_size), 4); + buffer.write(reinterpret_cast(&character_set), 1); + writeNulTerminatedString(username, buffer); + + if (capability_flags & CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA) + { + writeLengthEncodedString(auth_response, buffer); + } + else if (capability_flags & CLIENT_SECURE_CONNECTION) + { + writeString(auth_response, buffer); + } + else + { + writeNulTerminatedString(auth_response, buffer); + } + + if (capability_flags & CLIENT_CONNECT_WITH_DB) + { + writeNulTerminatedString(database, buffer); + } + + if (capability_flags & CLIENT_PLUGIN_AUTH) + { + writeNulTerminatedString(auth_plugin_name, buffer); + } + } + + void readPayloadImpl(ReadBuffer & payload) override { payload.readStrict(reinterpret_cast(&capability_flags), 4); payload.readStrict(reinterpret_cast(&max_packet_size), 4); @@ -573,12 +708,12 @@ protected: } }; -class AuthSwitchResponse : public LimitedClientPacket +class AuthSwitchResponse : public LimitedReadPacket { public: String value; - void readPayload(ReadBuffer & payload) override + void readPayloadImpl(ReadBuffer & payload) override { readStringUntilEOF(value, payload); } @@ -806,12 +941,12 @@ protected: } }; -class ComFieldList : public LimitedClientPacket +class ComFieldList : public LimitedReadPacket { public: String table, field_wildcard; - void readPayload(ReadBuffer & payload) override + void readPayloadImpl(ReadBuffer & payload) override { // Command byte has been already read from payload. readNullTerminated(table, payload); @@ -931,6 +1066,29 @@ public: } } + Native41(const String & password, const String & auth_plugin_data) + { + /// https://dev.mysql.com/doc/internals/en/secure-password-authentication.html + /// SHA1( password ) XOR SHA1( "20-bytes random data from server" SHA1( SHA1( password ) ) ) + Poco::SHA1Engine engine; + engine.update(password.data(), password.size()); + const Poco::SHA1Engine::Digest & password_sha1 = engine.digest(); + engine.update(password_sha1.data(), password_sha1.size()); + const Poco::SHA1Engine::Digest & password_double_sha1 = engine.digest(); + + engine.reset(); + + engine.update(auth_plugin_data.data(), auth_plugin_data.size()); + engine.update(password_double_sha1.data(), password_double_sha1.size()); + const Poco::SHA1Engine::Digest & digest = engine.digest(); + + scramble.resize(Poco::SHA1Engine::DIGEST_SIZE); + for (size_t i = 0; i < scramble.size(); i++) + { + scramble[i] = password_sha1[i] ^ digest[i]; + } + } + String getName() override { return "mysql_native_password"; diff --git a/src/Core/tests/CMakeLists.txt b/src/Core/tests/CMakeLists.txt index f7e51d92260..c1af57b5b85 100644 --- a/src/Core/tests/CMakeLists.txt +++ b/src/Core/tests/CMakeLists.txt @@ -15,3 +15,6 @@ if (ENABLE_FUZZING) add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp) target_link_libraries (names_and_types_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) endif () + +add_executable (mysql_client mysql_client.cpp) +target_link_libraries (mysql_client PRIVATE dbms) diff --git a/src/Core/tests/mysql_client.cpp b/src/Core/tests/mysql_client.cpp new file mode 100644 index 00000000000..66d1a73e5e7 --- /dev/null +++ b/src/Core/tests/mysql_client.cpp @@ -0,0 +1,13 @@ +#include + + +int main(int, char **) +{ + using namespace DB; + + UInt16 port = 3306; + String host = "127.0.0.1", user = "root", password = ""; + MySQLClient client(host, port, user, password, ""); + client.connect(); + return 0; +} From 8c4dc62ebbb4d00f5762865576b89b6efc455641 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Fri, 17 Apr 2020 09:47:04 +0800 Subject: [PATCH 021/374] make ReadPacket public and change variables in constructor function --- src/Core/MySQLClient.cpp | 4 ++-- src/Core/MySQLClient.h | 2 +- src/Core/MySQLProtocol.h | 32 ++++++++++++++++---------------- 3 files changed, 19 insertions(+), 19 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 882c7e83fd0..072465bc3e0 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -12,8 +12,8 @@ namespace ErrorCodes extern const int UNKNOWN_PACKET_FROM_SERVER; } -MySQLClient::MySQLClient(const String & _host, UInt16 _port, const String & _user, const String & _password, const String & _database) - : host(_host), port(_port), user(_user), password(_password), database(_database) +MySQLClient::MySQLClient(const String & host_, UInt16 port_, const String & user_, const String & password_, const String & database_) + : host(host_), port(port_), user(user_), password(password_), database(database_) { client_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_PLUGIN_AUTH | CLIENT_SECURE_CONNECTION; } diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 921626d05c7..73fe0d0ee8f 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -17,7 +17,7 @@ namespace DB class MySQLClient { public: - MySQLClient(const String & _host, UInt16 _port, const String & _user, const String & _password, const String & _database); + MySQLClient(const String & host_, UInt16 port_, const String & user_, const String & password_, const String & database_); void connect(); diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 55bdf987dee..cb3a2bc8186 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -434,7 +434,7 @@ size_t getLengthEncodedNumberSize(uint64_t x); size_t getLengthEncodedStringSize(const String & s); -class Handshake : public WritePacket, ReadPacket +class Handshake : public WritePacket, public ReadPacket { public: int protocol_version = 0xa; @@ -558,7 +558,7 @@ public: } }; -class HandshakeResponse : public WritePacket, ReadPacket +class HandshakeResponse : public WritePacket, public ReadPacket { public: uint32_t capability_flags = 0; @@ -572,20 +572,20 @@ public: HandshakeResponse() = default; HandshakeResponse( - UInt32 _capability_flags, - UInt32 _max_packet_size, - UInt8 _character_set, - const String & _username, - const String & _database, - const String & _auth_response, - const String & _auth_plugin_name) - : capability_flags(_capability_flags) - , max_packet_size(_max_packet_size) - , character_set(_character_set) - , username(_username) - , database(_database) - , auth_response(_auth_response) - , auth_plugin_name(_auth_plugin_name){}; + UInt32 capability_flags_, + UInt32 max_packet_size_, + UInt8 character_set_, + const String & username_, + const String & database_, + const String & auth_response_, + const String & auth_plugin_name_) + : capability_flags(capability_flags_) + , max_packet_size(max_packet_size_) + , character_set(character_set_) + , username(std::move(username_)) + , database(std::move(database_)) + , auth_response(std::move(auth_response_)) + , auth_plugin_name(std::move(auth_plugin_name_)){}; size_t getPayloadSize() const override { From 3b7a0f92a86adf287fce552bd58e38a7508e76ae Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 20 Apr 2020 09:35:00 +0800 Subject: [PATCH 022/374] fix the handshake packet --- src/Core/MySQLClient.cpp | 6 +++--- src/Core/MySQLClient.h | 5 +++-- src/Core/MySQLProtocol.h | 34 ++++++++++++++++++++------------- src/Core/tests/mysql_client.cpp | 4 ++-- 4 files changed, 29 insertions(+), 20 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 072465bc3e0..df4fabc512a 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -13,7 +13,7 @@ namespace ErrorCodes } MySQLClient::MySQLClient(const String & host_, UInt16 port_, const String & user_, const String & password_, const String & database_) - : host(host_), port(port_), user(user_), password(password_), database(database_) + : host(host_), port(port_), user(user_), password(std::move(password_)), database(std::move(database_)) { client_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_PLUGIN_AUTH | CLIENT_SECURE_CONNECTION; } @@ -73,9 +73,9 @@ void MySQLClient::handshake(ReadBuffer & payload) } Native41 native41(password, handshake.auth_plugin_data); - String response = native41.getAuthPluginData(); + String auth_plugin_data = native41.getAuthPluginData(); - HandshakeResponse handshakeResponse(client_capability_flags, 0, charset_utf8, user, database, handshake.auth_plugin_data, mysql_native_password); + HandshakeResponse handshakeResponse(client_capability_flags, max_packet_size, charset_utf8, user, database, auth_plugin_data, mysql_native_password); packet_sender->sendPacket(handshakeResponse, true); } } diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 73fe0d0ee8f..043f88e2710 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -1,6 +1,6 @@ #pragma once -#include #include +#include #include #include #include @@ -33,8 +33,9 @@ private: bool connected = false; UInt32 client_capability_flags = 0; - uint8_t seq = 0; + uint8_t seq = 1; UInt8 charset_utf8 = 33; + UInt32 max_packet_size = MySQLProtocol::MAX_PACKET_LENGTH; String mysql_native_password = "mysql_native_password"; std::shared_ptr in; diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index cb3a2bc8186..4699656ebba 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -496,7 +496,8 @@ public: buffer.readStrict(reinterpret_cast(&connection_id), 4); /// 8-bytes: auth-plugin-data-part-1 - buffer.readStrict(reinterpret_cast(&auth_plugin_data), AUTH_PLUGIN_DATA_PART_1_LENGTH); + auth_plugin_data.resize(AUTH_PLUGIN_DATA_PART_1_LENGTH); + buffer.readStrict(auth_plugin_data.data(), AUTH_PLUGIN_DATA_PART_1_LENGTH); /// 1-byte: [00] filler buffer.ignore(1); @@ -533,7 +534,8 @@ public: UInt8 part2_length = (auth_plugin_data_length - AUTH_PLUGIN_DATA_PART_1_LENGTH) > 13 ? 13 : (auth_plugin_data_length - AUTH_PLUGIN_DATA_PART_1_LENGTH); - buffer.readStrict((reinterpret_cast(&auth_plugin_data)) + AUTH_PLUGIN_DATA_PART_1_LENGTH, part2_length); + auth_plugin_data.resize(part2_length + AUTH_PLUGIN_DATA_PART_1_LENGTH); + buffer.readStrict(auth_plugin_data.data() + AUTH_PLUGIN_DATA_PART_1_LENGTH, part2_length); } if (capability_flags & MySQLProtocol::CLIENT_PLUGIN_AUTH) @@ -590,7 +592,9 @@ public: size_t getPayloadSize() const override { size_t size = 0; - size += 4 + 4 + 1; + size += 4 + 4 + 1 + 23; + size += username.size() + 1; + if (capability_flags & CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA) { size += getLengthEncodedStringSize(auth_response); @@ -619,14 +623,16 @@ public: buffer.write(reinterpret_cast(&capability_flags), 4); buffer.write(reinterpret_cast(&max_packet_size), 4); buffer.write(reinterpret_cast(&character_set), 1); - writeNulTerminatedString(username, buffer); + writeChar(0x0, 23, buffer); + writeNulTerminatedString(username, buffer); if (capability_flags & CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA) { writeLengthEncodedString(auth_response, buffer); } else if (capability_flags & CLIENT_SECURE_CONNECTION) { + writeChar(auth_response.size(), buffer); writeString(auth_response, buffer); } else @@ -1070,17 +1076,19 @@ public: { /// https://dev.mysql.com/doc/internals/en/secure-password-authentication.html /// SHA1( password ) XOR SHA1( "20-bytes random data from server" SHA1( SHA1( password ) ) ) - Poco::SHA1Engine engine; - engine.update(password.data(), password.size()); - const Poco::SHA1Engine::Digest & password_sha1 = engine.digest(); - engine.update(password_sha1.data(), password_sha1.size()); - const Poco::SHA1Engine::Digest & password_double_sha1 = engine.digest(); + Poco::SHA1Engine engine1; + engine1.update(password.data(), password.size()); + const Poco::SHA1Engine::Digest & password_sha1 = engine1.digest(); - engine.reset(); + Poco::SHA1Engine engine2; + engine2.update(password.data(), password.size()); + engine2.update(password_sha1.data(), password_sha1.size()); + const Poco::SHA1Engine::Digest & password_double_sha1 = engine2.digest(); - engine.update(auth_plugin_data.data(), auth_plugin_data.size()); - engine.update(password_double_sha1.data(), password_double_sha1.size()); - const Poco::SHA1Engine::Digest & digest = engine.digest(); + Poco::SHA1Engine engine3; + engine3.update(auth_plugin_data.data(), auth_plugin_data.size()); + engine3.update(password_double_sha1.data(), password_double_sha1.size()); + const Poco::SHA1Engine::Digest & digest = engine3.digest(); scramble.resize(Poco::SHA1Engine::DIGEST_SIZE); for (size_t i = 0; i < scramble.size(); i++) diff --git a/src/Core/tests/mysql_client.cpp b/src/Core/tests/mysql_client.cpp index 66d1a73e5e7..435df518dce 100644 --- a/src/Core/tests/mysql_client.cpp +++ b/src/Core/tests/mysql_client.cpp @@ -5,8 +5,8 @@ int main(int, char **) { using namespace DB; - UInt16 port = 3306; - String host = "127.0.0.1", user = "root", password = ""; + UInt16 port = 4407; + String host = "127.0.0.1", user = "root", password = "mock"; MySQLClient client(host, port, user, password, ""); client.connect(); return 0; From c1332834a944ea10f121f0bb118cf8025fe48777 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 20 Apr 2020 19:27:58 +0800 Subject: [PATCH 023/374] add ERR packet parse --- src/Core/MySQLClient.cpp | 22 +++++++++-- src/Core/MySQLClient.h | 4 +- src/Core/MySQLProtocol.h | 67 +++++++++++++++++++++++++++++---- src/Core/tests/mysql_client.cpp | 2 +- 4 files changed, 81 insertions(+), 14 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index df4fabc512a..0fbe3a5b067 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -36,7 +36,7 @@ void MySQLClient::connect() packet_sender = std::make_shared(*in, *out, seq); connected = true; - handshake(*in); + handshake(); } catch (Poco::Net::NetException & e) { @@ -61,10 +61,10 @@ void MySQLClient::close() } /// https://dev.mysql.com/doc/internals/en/connection-phase-packets.html -void MySQLClient::handshake(ReadBuffer & payload) +void MySQLClient::handshake() { Handshake handshake; - handshake.readPayloadImpl(payload); + packet_sender->receivePacket(handshake); if (handshake.auth_plugin_name != mysql_native_password) { throw Exception( @@ -75,7 +75,21 @@ void MySQLClient::handshake(ReadBuffer & payload) Native41 native41(password, handshake.auth_plugin_data); String auth_plugin_data = native41.getAuthPluginData(); - HandshakeResponse handshakeResponse(client_capability_flags, max_packet_size, charset_utf8, user, database, auth_plugin_data, mysql_native_password); + HandshakeResponse handshakeResponse( + client_capability_flags, max_packet_size, charset_utf8, user, database, auth_plugin_data, mysql_native_password); packet_sender->sendPacket(handshakeResponse, true); + + PacketResponse packetResponse; + packet_sender->receivePacket(packetResponse); + + switch (packetResponse.getType()) { + case PACKET_OK: + break; + case PACKET_ERR: + throw Exception(packetResponse.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); + break; + case PACKET_EOF: + break; + } } } diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 043f88e2710..813af4b91ba 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -33,7 +33,7 @@ private: bool connected = false; UInt32 client_capability_flags = 0; - uint8_t seq = 1; + uint8_t seq = 0; UInt8 charset_utf8 = 33; UInt32 max_packet_size = MySQLProtocol::MAX_PACKET_LENGTH; String mysql_native_password = "mysql_native_password"; @@ -43,7 +43,7 @@ private: std::unique_ptr socket; std::optional address; - void handshake(ReadBuffer & payload); + void handshake(); protected: std::shared_ptr packet_sender; diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 4699656ebba..ce62e111060 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -136,6 +136,12 @@ enum ColumnType MYSQL_TYPE_GEOMETRY = 0xff }; +enum ResponsePacketType +{ + PACKET_OK = 0x00, + PACKET_ERR = 0xff, + PACKET_EOF = 0xfe, +}; // https://dev.mysql.com/doc/dev/mysql-server/latest/group__group__cs__column__definition__flags.html enum ColumnDefinitionFlags @@ -237,6 +243,7 @@ public: virtual void readPayload(ReadBuffer & in, uint8_t & sequence_id) { PacketPayloadReadBuffer payload(in, sequence_id); + payload.next(); readPayloadImpl(payload); if (!payload.eof()) { @@ -484,8 +491,6 @@ public: void readPayloadImpl(ReadBuffer & buffer) override { - buffer.ignore(4); - /// 1-byte: [0a] protocol version buffer.readStrict(reinterpret_cast(&protocol_version), 1); @@ -587,7 +592,9 @@ public: , username(std::move(username_)) , database(std::move(database_)) , auth_response(std::move(auth_response_)) - , auth_plugin_name(std::move(auth_plugin_name_)){}; + , auth_plugin_name(std::move(auth_plugin_name_)) + { + } size_t getPayloadSize() const override { @@ -851,18 +858,19 @@ protected: } }; -class ERR_Packet : public WritePacket +class ERR_Packet : public WritePacket, public ReadPacket { - int error_code; +public: + int error_code = 0; String sql_state; String error_message; -public: + + ERR_Packet() = default; ERR_Packet(int error_code_, String sql_state_, String error_message_) : error_code(error_code_), sql_state(std::move(sql_state_)), error_message(std::move(error_message_)) { } -protected: size_t getPayloadSize() const override { return 4 + sql_state.length() + std::min(error_message.length(), MYSQL_ERRMSG_SIZE); @@ -876,6 +884,51 @@ protected: buffer.write(sql_state.data(), sql_state.length()); buffer.write(error_message.data(), std::min(error_message.length(), MYSQL_ERRMSG_SIZE)); } + + void readPayloadImpl(ReadBuffer & payload) override + { + UInt8 header = 0; + payload.readStrict(reinterpret_cast(&header), 1); + assert(header == 0xff); + payload.readStrict(reinterpret_cast(&error_code), 2); + payload.ignore(1); + payload.readStrict(reinterpret_cast(&sql_state), 5); + readString(error_message, payload); + } +}; + +/// https://dev.mysql.com/doc/internals/en/generic-response-packets.html +class PacketResponse : public ReadPacket +{ +public: + OK_Packet * ok; + ERR_Packet err; + EOF_Packet * eof; + + PacketResponse() = default; + + void readPayloadImpl(ReadBuffer & payload) override + { + UInt8 header = *payload.position(); + switch (header) + { + case PACKET_OK: + packetType = PACKET_OK; + break; + case PACKET_ERR: + packetType = PACKET_ERR; + err.readPayloadImpl(payload); + break; + case PACKET_EOF: + packetType = PACKET_EOF; + break; + }; + } + + ResponsePacketType getType() { return packetType; } + +private: + ResponsePacketType packetType = PACKET_OK; }; class ColumnDefinition : public WritePacket diff --git a/src/Core/tests/mysql_client.cpp b/src/Core/tests/mysql_client.cpp index 435df518dce..eaf85ac5b5b 100644 --- a/src/Core/tests/mysql_client.cpp +++ b/src/Core/tests/mysql_client.cpp @@ -6,7 +6,7 @@ int main(int, char **) using namespace DB; UInt16 port = 4407; - String host = "127.0.0.1", user = "root", password = "mock"; + String host = "127.0.0.1", user = "mock", password = "mock"; MySQLClient client(host, port, user, password, ""); client.connect(); return 0; From ab8b847e66aadda369166bdc2b5047610a4a32bb Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 20 Apr 2020 22:31:43 +0800 Subject: [PATCH 024/374] add OK/ERR packet parse --- src/Core/MySQLClient.cpp | 5 +-- src/Core/MySQLProtocol.h | 76 +++++++++++++++++++++++++++++++++------- 2 files changed, 67 insertions(+), 14 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 0fbe3a5b067..44648551622 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -79,10 +79,11 @@ void MySQLClient::handshake() client_capability_flags, max_packet_size, charset_utf8, user, database, auth_plugin_data, mysql_native_password); packet_sender->sendPacket(handshakeResponse, true); - PacketResponse packetResponse; + PacketResponse packetResponse(handshake.capability_flags); packet_sender->receivePacket(packetResponse); - switch (packetResponse.getType()) { + switch (packetResponse.getType()) + { case PACKET_OK: break; case PACKET_ERR: diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index ce62e111060..08f764e7b11 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -752,17 +752,21 @@ protected: }; -class OK_Packet : public WritePacket +class OK_Packet : public WritePacket, public ReadPacket { +public: uint8_t header; uint32_t capabilities; uint64_t affected_rows; + uint64_t last_insert_id; int16_t warnings = 0; uint32_t status_flags; String session_state_changes; String info; -public: - OK_Packet(uint8_t header_, + + OK_Packet(uint32_t capabilities_) : capabilities(capabilities_) { } + OK_Packet( + uint8_t header_, uint32_t capabilities_, uint64_t affected_rows_, uint32_t status_flags_, @@ -779,7 +783,6 @@ public: { } -protected: size_t getPayloadSize() const override { size_t result = 2 + getLengthEncodedNumberSize(affected_rows); @@ -834,17 +837,51 @@ protected: writeString(info, buffer); } } + + void readPayloadImpl(ReadBuffer & payload) override + { + payload.readStrict(reinterpret_cast(&header), 1); + affected_rows = readLengthEncodedNumber(payload); + last_insert_id = readLengthEncodedNumber(payload); + + if (capabilities & CLIENT_PROTOCOL_41) + { + payload.readStrict(reinterpret_cast(&status_flags), 2); + payload.readStrict(reinterpret_cast(&warnings), 2); + } + else if (capabilities & CLIENT_TRANSACTIONS) + { + payload.readStrict(reinterpret_cast(&status_flags), 2); + } + + if (capabilities & CLIENT_SESSION_TRACK) + { + auto len = readLengthEncodedNumber(payload); + info.resize(len); + payload.readStrict(info.data(), len); + if (status_flags & SERVER_SESSION_STATE_CHANGED) { + len = readLengthEncodedNumber(payload); + session_state_changes.resize(len); + payload.readStrict(session_state_changes.data(), len); + } + } + else + { + readString(info, payload); + } + } }; -class EOF_Packet : public WritePacket +class EOF_Packet : public WritePacket, public ReadPacket { +public: int warnings; int status_flags; -public: + + EOF_Packet() = default; EOF_Packet(int warnings_, int status_flags_) : warnings(warnings_), status_flags(status_flags_) {} -protected: size_t getPayloadSize() const override { return 5; @@ -856,6 +893,15 @@ protected: buffer.write(reinterpret_cast(&warnings), 2); buffer.write(reinterpret_cast(&status_flags), 2); } + + void readPayloadImpl(ReadBuffer & payload) override + { + UInt8 header = 0; + payload.readStrict(reinterpret_cast(&header), 1); + assert(header == 0xfe); + payload.readStrict(reinterpret_cast(&warnings), 2); + payload.readStrict(reinterpret_cast(&status_flags), 2); + } }; class ERR_Packet : public WritePacket, public ReadPacket @@ -890,10 +936,14 @@ public: UInt8 header = 0; payload.readStrict(reinterpret_cast(&header), 1); assert(header == 0xff); + payload.readStrict(reinterpret_cast(&error_code), 2); payload.ignore(1); + + sql_state.resize(5); payload.readStrict(reinterpret_cast(&sql_state), 5); - readString(error_message, payload); + + readStringUntilEOF(error_message, payload); } }; @@ -901,11 +951,11 @@ public: class PacketResponse : public ReadPacket { public: - OK_Packet * ok; - ERR_Packet err; - EOF_Packet * eof; + OK_Packet ok; + ERR_Packet err; + EOF_Packet eof; - PacketResponse() = default; + PacketResponse(UInt32 server_capability_flags_) : ok(OK_Packet(server_capability_flags_)) { } void readPayloadImpl(ReadBuffer & payload) override { @@ -914,6 +964,7 @@ public: { case PACKET_OK: packetType = PACKET_OK; + ok.readPayloadImpl(payload); break; case PACKET_ERR: packetType = PACKET_ERR; @@ -921,6 +972,7 @@ public: break; case PACKET_EOF: packetType = PACKET_EOF; + eof.readPayloadImpl(payload); break; }; } From 1a9118d72233ae4369abd1041982c5095eec4542 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 20 Apr 2020 23:06:17 +0800 Subject: [PATCH 025/374] add error to MySQL client connect --- src/Core/MySQLClient.cpp | 64 +++++++++++++-------------------- src/Core/MySQLClient.h | 14 ++++---- src/Core/MySQLProtocol.h | 27 +++----------- src/Core/tests/mysql_client.cpp | 10 ++++-- 4 files changed, 44 insertions(+), 71 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 44648551622..1ebffefdd1e 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -2,13 +2,10 @@ namespace DB { -using namespace MySQLProtocol; using namespace MySQLProtocol::Authentication; namespace ErrorCodes { - extern const int NETWORK_ERROR; - extern const int SOCKET_TIMEOUT; extern const int UNKNOWN_PACKET_FROM_SERVER; } @@ -18,36 +15,22 @@ MySQLClient::MySQLClient(const String & host_, UInt16 port_, const String & user client_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_PLUGIN_AUTH | CLIENT_SECURE_CONNECTION; } -void MySQLClient::connect() +bool MySQLClient::connect() { - try - { - if (connected) - { - close(); - } - - socket = std::make_unique(); - address = DNSResolver::instance().resolveAddress(host, port); - socket->connect(*address); - - in = std::make_shared(*socket); - out = std::make_shared(*socket); - packet_sender = std::make_shared(*in, *out, seq); - connected = true; - - handshake(); - } - catch (Poco::Net::NetException & e) + if (connected) { close(); - throw NetException(e.displayText(), ErrorCodes::NETWORK_ERROR); - } - catch (Poco::TimeoutException & e) - { - close(); - throw NetException(e.displayText(), ErrorCodes::SOCKET_TIMEOUT); } + + socket = std::make_unique(); + address = DNSResolver::instance().resolveAddress(host, port); + socket->connect(*address); + + in = std::make_shared(*socket); + out = std::make_shared(*socket); + packet_sender = std::make_shared(*in, *out, seq); + connected = true; + return handshake(); } void MySQLClient::close() @@ -61,7 +44,7 @@ void MySQLClient::close() } /// https://dev.mysql.com/doc/internals/en/connection-phase-packets.html -void MySQLClient::handshake() +bool MySQLClient::handshake() { Handshake handshake; packet_sender->receivePacket(handshake); @@ -81,16 +64,19 @@ void MySQLClient::handshake() PacketResponse packetResponse(handshake.capability_flags); packet_sender->receivePacket(packetResponse); - - switch (packetResponse.getType()) + if (packetResponse.getType() != PACKET_ERR) { - case PACKET_OK: - break; - case PACKET_ERR: - throw Exception(packetResponse.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); - break; - case PACKET_EOF: - break; + return true; + } + else + { + last_error = packetResponse.err.error_message; + return false; } } + +String MySQLClient::error() +{ + return last_error; +} } diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 813af4b91ba..3d878df5b34 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -14,14 +14,15 @@ namespace DB { +using namespace MySQLProtocol; + class MySQLClient { public: MySQLClient(const String & host_, UInt16 port_, const String & user_, const String & password_, const String & database_); - - void connect(); - + bool connect(); void close(); + String error(); private: String host; @@ -32,6 +33,7 @@ private: bool connected = false; UInt32 client_capability_flags = 0; + String last_error; uint8_t seq = 0; UInt8 charset_utf8 = 33; @@ -42,10 +44,8 @@ private: std::shared_ptr out; std::unique_ptr socket; std::optional address; + std::shared_ptr packet_sender; - void handshake(); - -protected: - std::shared_ptr packet_sender; + bool handshake(); }; } diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 08f764e7b11..e907069cd0c 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -491,49 +491,30 @@ public: void readPayloadImpl(ReadBuffer & buffer) override { - /// 1-byte: [0a] protocol version buffer.readStrict(reinterpret_cast(&protocol_version), 1); - - /// string[NUL]: server version readNullTerminated(server_version, buffer); - - /// 4-bytes: connection id buffer.readStrict(reinterpret_cast(&connection_id), 4); - /// 8-bytes: auth-plugin-data-part-1 auth_plugin_data.resize(AUTH_PLUGIN_DATA_PART_1_LENGTH); buffer.readStrict(auth_plugin_data.data(), AUTH_PLUGIN_DATA_PART_1_LENGTH); - /// 1-byte: [00] filler buffer.ignore(1); - - /// 2-bytes: capability flags lower 2-bytes buffer.readStrict(reinterpret_cast(&capability_flags), 2); - - /// 1-byte: character set buffer.readStrict(reinterpret_cast(&character_set), 1); - - /// 2-bytes: status flags(ignored) buffer.readStrict(reinterpret_cast(&status_flags), 2); - - /// 2-bytes: capability flags upper 2-bytes buffer.readStrict((reinterpret_cast(&capability_flags)) + 2, 2); UInt8 auth_plugin_data_length = 0; if (capability_flags & MySQLProtocol::CLIENT_PLUGIN_AUTH) { - /// 1-byte: length of auth-plugin-data buffer.readStrict(reinterpret_cast(&auth_plugin_data_length), 1); } else { - /// 1-byte: [00] buffer.ignore(1); } - /// string[10] reserved (all [00]) buffer.ignore(10); - if (capability_flags & MySQLProtocol::CLIENT_SECURE_CONNECTION) { UInt8 part2_length = (auth_plugin_data_length - AUTH_PLUGIN_DATA_PART_1_LENGTH) > 13 @@ -764,7 +745,7 @@ public: String session_state_changes; String info; - OK_Packet(uint32_t capabilities_) : capabilities(capabilities_) { } + OK_Packet(uint32_t capabilities_) : header(0x00), capabilities(capabilities_), affected_rows(0), last_insert_id(0), status_flags(0) { } OK_Packet( uint8_t header_, uint32_t capabilities_, @@ -776,6 +757,7 @@ public: : header(header_) , capabilities(capabilities_) , affected_rows(affected_rows_) + , last_insert_id(0) , warnings(warnings_) , status_flags(status_flags_) , session_state_changes(std::move(session_state_changes_)) @@ -814,7 +796,7 @@ public: { buffer.write(header); writeLengthEncodedNumber(affected_rows, buffer); - writeLengthEncodedNumber(0, buffer); /// last insert-id + writeLengthEncodedNumber(last_insert_id, buffer); /// last insert-id if (capabilities & CLIENT_PROTOCOL_41) { @@ -859,7 +841,8 @@ public: auto len = readLengthEncodedNumber(payload); info.resize(len); payload.readStrict(info.data(), len); - if (status_flags & SERVER_SESSION_STATE_CHANGED) { + if (status_flags & SERVER_SESSION_STATE_CHANGED) + { len = readLengthEncodedNumber(payload); session_state_changes.resize(len); payload.readStrict(session_state_changes.data(), len); diff --git a/src/Core/tests/mysql_client.cpp b/src/Core/tests/mysql_client.cpp index eaf85ac5b5b..bce50befaa0 100644 --- a/src/Core/tests/mysql_client.cpp +++ b/src/Core/tests/mysql_client.cpp @@ -1,3 +1,4 @@ +#include #include @@ -5,9 +6,12 @@ int main(int, char **) { using namespace DB; - UInt16 port = 4407; - String host = "127.0.0.1", user = "mock", password = "mock"; + UInt16 port = 9001; + String host = "127.0.0.1", user = "default", password = "123"; MySQLClient client(host, port, user, password, ""); - client.connect(); + if (!client.connect()) + { + std::cerr << "Connect Error: " << client.error() << std::endl; + } return 0; } From f5a53dd27085ef465e05d8195199ee7de44b78a7 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 21 Apr 2020 13:41:49 +0800 Subject: [PATCH 026/374] handshake auth ok --- src/Core/MySQLProtocol.h | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index e907069cd0c..643168018a0 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -520,8 +520,9 @@ public: UInt8 part2_length = (auth_plugin_data_length - AUTH_PLUGIN_DATA_PART_1_LENGTH) > 13 ? 13 : (auth_plugin_data_length - AUTH_PLUGIN_DATA_PART_1_LENGTH); - auth_plugin_data.resize(part2_length + AUTH_PLUGIN_DATA_PART_1_LENGTH); - buffer.readStrict(auth_plugin_data.data() + AUTH_PLUGIN_DATA_PART_1_LENGTH, part2_length); + auth_plugin_data.resize(part2_length + AUTH_PLUGIN_DATA_PART_1_LENGTH - 1); + buffer.readStrict(auth_plugin_data.data() + AUTH_PLUGIN_DATA_PART_1_LENGTH, part2_length - 1); + buffer.ignore(1); } if (capability_flags & MySQLProtocol::CLIENT_PLUGIN_AUTH) @@ -621,7 +622,7 @@ public: else if (capability_flags & CLIENT_SECURE_CONNECTION) { writeChar(auth_response.size(), buffer); - writeString(auth_response, buffer); + writeString(auth_response.data(), auth_response.size(), buffer); } else { @@ -1165,11 +1166,10 @@ public: /// https://dev.mysql.com/doc/internals/en/secure-password-authentication.html /// SHA1( password ) XOR SHA1( "20-bytes random data from server" SHA1( SHA1( password ) ) ) Poco::SHA1Engine engine1; - engine1.update(password.data(), password.size()); + engine1.update(password.data()); const Poco::SHA1Engine::Digest & password_sha1 = engine1.digest(); Poco::SHA1Engine engine2; - engine2.update(password.data(), password.size()); engine2.update(password_sha1.data(), password_sha1.size()); const Poco::SHA1Engine::Digest & password_double_sha1 = engine2.digest(); @@ -1178,10 +1178,10 @@ public: engine3.update(password_double_sha1.data(), password_double_sha1.size()); const Poco::SHA1Engine::Digest & digest = engine3.digest(); - scramble.resize(Poco::SHA1Engine::DIGEST_SIZE); - for (size_t i = 0; i < scramble.size(); i++) + scramble.resize(SCRAMBLE_LENGTH); + for (size_t i = 0; i < SCRAMBLE_LENGTH; i++) { - scramble[i] = password_sha1[i] ^ digest[i]; + scramble[i] = static_cast(password_sha1[i] ^ digest[i]); } } From b9e2c0d72c684f45e706037567623be33dbc0013 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Wed, 22 Apr 2020 10:27:06 +0800 Subject: [PATCH 027/374] add gtest for MySQL Protocol handshake --- src/Core/MySQLProtocol.h | 10 ++++----- src/Core/tests/gtest_MySQLProtocol.cpp | 31 ++++++++++++++++++++++++++ 2 files changed, 35 insertions(+), 6 deletions(-) create mode 100644 src/Core/tests/gtest_MySQLProtocol.cpp diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 643168018a0..35c4c3da1b9 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -517,11 +517,9 @@ public: buffer.ignore(10); if (capability_flags & MySQLProtocol::CLIENT_SECURE_CONNECTION) { - UInt8 part2_length = (auth_plugin_data_length - AUTH_PLUGIN_DATA_PART_1_LENGTH) > 13 - ? 13 - : (auth_plugin_data_length - AUTH_PLUGIN_DATA_PART_1_LENGTH); - auth_plugin_data.resize(part2_length + AUTH_PLUGIN_DATA_PART_1_LENGTH - 1); - buffer.readStrict(auth_plugin_data.data() + AUTH_PLUGIN_DATA_PART_1_LENGTH, part2_length - 1); + UInt8 part2_length = (SCRAMBLE_LENGTH - AUTH_PLUGIN_DATA_PART_1_LENGTH); + auth_plugin_data.resize(SCRAMBLE_LENGTH); + buffer.readStrict(auth_plugin_data.data() + AUTH_PLUGIN_DATA_PART_1_LENGTH, part2_length); buffer.ignore(1); } @@ -958,7 +956,7 @@ public: packetType = PACKET_EOF; eof.readPayloadImpl(payload); break; - }; + } } ResponsePacketType getType() { return packetType; } diff --git a/src/Core/tests/gtest_MySQLProtocol.cpp b/src/Core/tests/gtest_MySQLProtocol.cpp new file mode 100644 index 00000000000..f29bd8738f7 --- /dev/null +++ b/src/Core/tests/gtest_MySQLProtocol.cpp @@ -0,0 +1,31 @@ +#include +#include + +#include +#include +#include + +using namespace DB; +using namespace MySQLProtocol; + +TEST(MySQLProtocol, Handshake) +{ + UInt32 server_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_SECURE_CONNECTION | CLIENT_PLUGIN_AUTH + | CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA | CLIENT_CONNECT_WITH_DB | CLIENT_DEPRECATE_EOF; + + std::string s; + WriteBufferFromString out(s); + Handshake server_handshake(server_capability_flags, 0, "ClickHouse", "mysql_native_password", "aaaaaaaaaaaaaaaaaaaaa"); + server_handshake.writePayloadImpl(out); + + ReadBufferFromString in(s); + Handshake client_handshake; + client_handshake.readPayloadImpl(in); + + EXPECT_EQ(server_handshake.capability_flags, client_handshake.capability_flags); + EXPECT_EQ(server_handshake.status_flags, client_handshake.status_flags); + EXPECT_EQ(server_handshake.server_version, client_handshake.server_version); + EXPECT_EQ(server_handshake.protocol_version, client_handshake.protocol_version); + EXPECT_EQ(server_handshake.auth_plugin_data.substr(0, 20), client_handshake.auth_plugin_data); + EXPECT_EQ(server_handshake.auth_plugin_name, client_handshake.auth_plugin_name); +} From 6cac6a4f7687ec511f423b5d799f00ee1c3469eb Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Wed, 22 Apr 2020 15:26:12 +0800 Subject: [PATCH 028/374] add mysql protocol test --- src/Core/MySQLClient.cpp | 12 +- src/Core/MySQLProtocol.h | 18 +-- src/Core/tests/CMakeLists.txt | 3 + src/Core/tests/gtest_MySQLProtocol.cpp | 31 ------ src/Core/tests/mysql_client.cpp | 17 --- src/Core/tests/mysql_protocol.cpp | 146 +++++++++++++++++++++++++ 6 files changed, 164 insertions(+), 63 deletions(-) delete mode 100644 src/Core/tests/gtest_MySQLProtocol.cpp delete mode 100644 src/Core/tests/mysql_client.cpp create mode 100644 src/Core/tests/mysql_protocol.cpp diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 1ebffefdd1e..430ee902143 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -58,19 +58,19 @@ bool MySQLClient::handshake() Native41 native41(password, handshake.auth_plugin_data); String auth_plugin_data = native41.getAuthPluginData(); - HandshakeResponse handshakeResponse( + HandshakeResponse handshake_response( client_capability_flags, max_packet_size, charset_utf8, user, database, auth_plugin_data, mysql_native_password); - packet_sender->sendPacket(handshakeResponse, true); + packet_sender->sendPacket(handshake_response, true); - PacketResponse packetResponse(handshake.capability_flags); - packet_sender->receivePacket(packetResponse); - if (packetResponse.getType() != PACKET_ERR) + PacketResponse packet_response(handshake.capability_flags); + packet_sender->receivePacket(packet_response); + if (packet_response.getType() != PACKET_ERR) { return true; } else { - last_error = packetResponse.err.error_message; + last_error = packet_response.err.error_message; return false; } } diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 35c4c3da1b9..979b5beef32 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -857,8 +857,9 @@ public: class EOF_Packet : public WritePacket, public ReadPacket { public: - int warnings; - int status_flags; + UInt8 header = 0xfe; + int warnings = 0; + int status_flags = 0; EOF_Packet() = default; EOF_Packet(int warnings_, int status_flags_) : warnings(warnings_), status_flags(status_flags_) @@ -871,14 +872,13 @@ public: void writePayloadImpl(WriteBuffer & buffer) const override { - buffer.write(0xfe); // EOF header + buffer.write(header); // EOF header buffer.write(reinterpret_cast(&warnings), 2); buffer.write(reinterpret_cast(&status_flags), 2); } void readPayloadImpl(ReadBuffer & payload) override { - UInt8 header = 0; payload.readStrict(reinterpret_cast(&header), 1); assert(header == 0xfe); payload.readStrict(reinterpret_cast(&warnings), 2); @@ -889,6 +889,7 @@ public: class ERR_Packet : public WritePacket, public ReadPacket { public: + UInt8 header = 0xff; int error_code = 0; String sql_state; String error_message; @@ -906,7 +907,7 @@ public: void writePayloadImpl(WriteBuffer & buffer) const override { - buffer.write(0xff); + buffer.write(header); buffer.write(reinterpret_cast(&error_code), 2); buffer.write('#'); buffer.write(sql_state.data(), sql_state.length()); @@ -915,7 +916,6 @@ public: void readPayloadImpl(ReadBuffer & payload) override { - UInt8 header = 0; payload.readStrict(reinterpret_cast(&header), 1); assert(header == 0xff); @@ -923,9 +923,9 @@ public: payload.ignore(1); sql_state.resize(5); - payload.readStrict(reinterpret_cast(&sql_state), 5); + payload.readStrict(reinterpret_cast(sql_state.data()), 5); - readStringUntilEOF(error_message, payload); + readNullTerminated(error_message, payload); } }; @@ -941,7 +941,7 @@ public: void readPayloadImpl(ReadBuffer & payload) override { - UInt8 header = *payload.position(); + UInt8 header = static_cast(*payload.position()); switch (header) { case PACKET_OK: diff --git a/src/Core/tests/CMakeLists.txt b/src/Core/tests/CMakeLists.txt index c1af57b5b85..a5d694b7358 100644 --- a/src/Core/tests/CMakeLists.txt +++ b/src/Core/tests/CMakeLists.txt @@ -18,3 +18,6 @@ endif () add_executable (mysql_client mysql_client.cpp) target_link_libraries (mysql_client PRIVATE dbms) + +add_executable (mysql_protocol mysql_protocol.cpp) +target_link_libraries (mysql_protocol PRIVATE dbms) diff --git a/src/Core/tests/gtest_MySQLProtocol.cpp b/src/Core/tests/gtest_MySQLProtocol.cpp deleted file mode 100644 index f29bd8738f7..00000000000 --- a/src/Core/tests/gtest_MySQLProtocol.cpp +++ /dev/null @@ -1,31 +0,0 @@ -#include -#include - -#include -#include -#include - -using namespace DB; -using namespace MySQLProtocol; - -TEST(MySQLProtocol, Handshake) -{ - UInt32 server_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_SECURE_CONNECTION | CLIENT_PLUGIN_AUTH - | CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA | CLIENT_CONNECT_WITH_DB | CLIENT_DEPRECATE_EOF; - - std::string s; - WriteBufferFromString out(s); - Handshake server_handshake(server_capability_flags, 0, "ClickHouse", "mysql_native_password", "aaaaaaaaaaaaaaaaaaaaa"); - server_handshake.writePayloadImpl(out); - - ReadBufferFromString in(s); - Handshake client_handshake; - client_handshake.readPayloadImpl(in); - - EXPECT_EQ(server_handshake.capability_flags, client_handshake.capability_flags); - EXPECT_EQ(server_handshake.status_flags, client_handshake.status_flags); - EXPECT_EQ(server_handshake.server_version, client_handshake.server_version); - EXPECT_EQ(server_handshake.protocol_version, client_handshake.protocol_version); - EXPECT_EQ(server_handshake.auth_plugin_data.substr(0, 20), client_handshake.auth_plugin_data); - EXPECT_EQ(server_handshake.auth_plugin_name, client_handshake.auth_plugin_name); -} diff --git a/src/Core/tests/mysql_client.cpp b/src/Core/tests/mysql_client.cpp deleted file mode 100644 index bce50befaa0..00000000000 --- a/src/Core/tests/mysql_client.cpp +++ /dev/null @@ -1,17 +0,0 @@ -#include -#include - - -int main(int, char **) -{ - using namespace DB; - - UInt16 port = 9001; - String host = "127.0.0.1", user = "default", password = "123"; - MySQLClient client(host, port, user, password, ""); - if (!client.connect()) - { - std::cerr << "Connect Error: " << client.error() << std::endl; - } - return 0; -} diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp new file mode 100644 index 00000000000..870eb4966c6 --- /dev/null +++ b/src/Core/tests/mysql_protocol.cpp @@ -0,0 +1,146 @@ +#include + +#include +#include +#include + + +int main(int, char **) +{ + using namespace DB; + using namespace MySQLProtocol; + using namespace MySQLProtocol::Authentication; + + /* + UInt16 port = 9001; + String host = "127.0.0.1", user = "default", password = "123"; + MySQLClient client(host, port, user, password, ""); + if (!client.connect()) + { + std::cerr << "Connect Error: " << client.error() << std::endl; + } + */ + String user = "default"; + String password = "123"; + String database = ""; + + UInt8 charset_utf8 = 33; + UInt32 max_packet_size = MySQLProtocol::MAX_PACKET_LENGTH; + String mysql_native_password = "mysql_native_password"; + + UInt32 server_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_SECURE_CONNECTION | CLIENT_PLUGIN_AUTH + | CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA | CLIENT_CONNECT_WITH_DB | CLIENT_DEPRECATE_EOF; + + UInt32 client_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_PLUGIN_AUTH | CLIENT_SECURE_CONNECTION; + + /// Handshake packet + { + /// 1. Greeting: + /// 1.1 Server writes greeting to client + std::string s0; + WriteBufferFromString out0(s0); + + Handshake server_handshake(server_capability_flags, -1, "ClickHouse", "mysql_native_password", "aaaaaaaaaaaaaaaaaaaaa"); + server_handshake.writePayloadImpl(out0); + + /// 1.2 Client reads the greeting + ReadBufferFromString in0(s0); + Handshake client_handshake; + client_handshake.readPayloadImpl(in0); + + /// Check packet + ASSERT(server_handshake.capability_flags == client_handshake.capability_flags); + ASSERT(server_handshake.status_flags == client_handshake.status_flags); + ASSERT(server_handshake.server_version == client_handshake.server_version); + ASSERT(server_handshake.protocol_version == client_handshake.protocol_version); + ASSERT(server_handshake.auth_plugin_data.substr(0, 20) == client_handshake.auth_plugin_data); + ASSERT(server_handshake.auth_plugin_name == client_handshake.auth_plugin_name); + + /// 2. Greeting Response: + std::string s1; + WriteBufferFromString out1(s1); + + /// 2.1 Client writes to server + Native41 native41(password, client_handshake.auth_plugin_data); + String auth_plugin_data = native41.getAuthPluginData(); + HandshakeResponse client_handshake_response( + client_capability_flags, max_packet_size, charset_utf8, user, database, auth_plugin_data, mysql_native_password); + client_handshake_response.writePayloadImpl(out1); + + /// 2.2 Server reads the response + ReadBufferFromString in1(s1); + HandshakeResponse server_handshake_response; + server_handshake_response.readPayloadImpl(in1); + + /// Check + ASSERT(server_handshake_response.capability_flags == client_handshake_response.capability_flags); + ASSERT(server_handshake_response.character_set == client_handshake_response.character_set); + ASSERT(server_handshake_response.username == client_handshake_response.username); + ASSERT(server_handshake_response.database == client_handshake_response.database); + ASSERT(server_handshake_response.auth_response == client_handshake_response.auth_response); + ASSERT(server_handshake_response.auth_plugin_name == client_handshake_response.auth_plugin_name); + } + + /// OK Packet + { + // 1. Server writes packet + std::string s0; + WriteBufferFromString out0(s0); + OK_Packet server(0x00, server_capability_flags, 0, 0, 0, "", ""); + server.writePayloadImpl(out0); + + // 2. Client reads packet + ReadBufferFromString in1(s0); + PacketResponse client(server_capability_flags); + client.readPayloadImpl(in1); + + // Check + ASSERT(client.getType() == PACKET_OK); + ASSERT(client.ok.header == server.header); + ASSERT(client.ok.status_flags == server.status_flags); + ASSERT(client.ok.capabilities == server.capabilities); + } + + /// ERR Packet + { + // 1. Server writes packet + std::string s0; + WriteBufferFromString out0(s0); + ERR_Packet server(123, "12345", "This is the error message"); + server.writePayloadImpl(out0); + + // 2. Client reads packet + ReadBufferFromString in1(s0); + PacketResponse client(server_capability_flags); + client.readPayloadImpl(in1); + + // Check + ASSERT(client.getType() == PACKET_ERR); + ASSERT(client.err.header == server.header); + ASSERT(client.err.error_code == server.error_code); + ASSERT(client.err.sql_state == server.sql_state); + ASSERT(client.err.error_message == server.error_message); + } + + /// EOF Packet + { + // 1. Server writes packet + std::string s0; + WriteBufferFromString out0(s0); + EOF_Packet server(1, 1); + server.writePayloadImpl(out0); + + // 2. Client reads packet + ReadBufferFromString in1(s0); + PacketResponse client(server_capability_flags); + client.readPayloadImpl(in1); + + // Check + ASSERT(client.getType() == PACKET_EOF); + ASSERT(client.eof.header == server.header); + ASSERT(client.eof.warnings == server.warnings); + ASSERT(client.eof.status_flags == server.status_flags); + } + + return 0; +} From b23816799c2b8a93f0c024f52c99f6f381c01e5f Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Thu, 23 Apr 2020 16:35:02 +0800 Subject: [PATCH 029/374] add read columns --- src/Core/MySQLClient.cpp | 65 ++++++++++++++++--- src/Core/MySQLClient.h | 30 ++++++++- src/Core/MySQLProtocol.h | 103 ++++++++++++++++++++---------- src/Core/tests/mysql_protocol.cpp | 77 +++++++++++++++++----- 4 files changed, 213 insertions(+), 62 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 430ee902143..ad7f7b66c63 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -19,7 +19,7 @@ bool MySQLClient::connect() { if (connected) { - close(); + disconnect(); } socket = std::make_unique(); @@ -33,7 +33,7 @@ bool MySQLClient::connect() return handshake(); } -void MySQLClient::close() +void MySQLClient::disconnect() { in = nullptr; out = nullptr; @@ -54,6 +54,7 @@ bool MySQLClient::handshake() "Only support " + mysql_native_password + " auth plugin name, but got " + handshake.auth_plugin_name, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); } + server_capability_flags = handshake.capability_flags; Native41 native41(password, handshake.auth_plugin_data); String auth_plugin_data = native41.getAuthPluginData(); @@ -62,21 +63,67 @@ bool MySQLClient::handshake() client_capability_flags, max_packet_size, charset_utf8, user, database, auth_plugin_data, mysql_native_password); packet_sender->sendPacket(handshake_response, true); - PacketResponse packet_response(handshake.capability_flags); + PacketResponse packet_response(server_capability_flags); packet_sender->receivePacket(packet_response); - if (packet_response.getType() != PACKET_ERR) - { - return true; - } - else + packet_sender->resetSequenceId(); + if (packet_response.getType() == PACKET_ERR) { last_error = packet_response.err.error_message; - return false; } + return (packet_response.getType() != PACKET_ERR); +} + +bool MySQLClient::ping() +{ + return writeCommand(Command::COM_PING, ""); +} + +bool MySQLClient::initdb(String db) +{ + return writeCommand(Command::COM_INIT_DB, db); +} + +bool MySQLClient::query(String q) +{ + return writeCommand(Command::COM_QUERY, q); } String MySQLClient::error() { return last_error; } + +bool MySQLClient::writeCommand(char command, String query) +{ + bool ret = false; + + WriteCommand write_command(command, query); + packet_sender->sendPacket(write_command, true); + + PacketResponse packet_response(server_capability_flags); + packet_sender->receivePacket(packet_response); + switch (packet_response.getType()) + { + case PACKET_ERR: + last_error = packet_response.err.error_message; + break; + case PACKET_OK: + ret = readColumns(packet_response.column_length); + break; + default: + break; + } + packet_sender->resetSequenceId(); + return ret; +} + +bool MySQLClient::readColumns(int column_length) +{ + for (auto i = 0; i < column_length; i++) + { + ColumnDefinition cd; + packet_sender->receivePacket(cd); + } + return true; +} } diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 3d878df5b34..2140f1032a7 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -21,7 +21,10 @@ class MySQLClient public: MySQLClient(const String & host_, UInt16 port_, const String & user_, const String & password_, const String & database_); bool connect(); - void close(); + void disconnect(); + bool ping(); + bool initdb(String db); + bool query(String q); String error(); private: @@ -32,8 +35,9 @@ private: String database; bool connected = false; - UInt32 client_capability_flags = 0; String last_error; + UInt32 client_capability_flags = 0; + UInt32 server_capability_flags = 0; uint8_t seq = 0; UInt8 charset_utf8 = 33; @@ -47,5 +51,27 @@ private: std::shared_ptr packet_sender; bool handshake(); + bool readColumns(int column_length); + bool writeCommand(char command, String query); +}; + +class WriteCommand : public WritePacket +{ +public: + char command; + String query; + + WriteCommand(char command_, String query_) : command(command_), query(query_) { } + + size_t getPayloadSize() const override { return 1 + query.size(); } + + void writePayloadImpl(WriteBuffer & buffer) const override + { + buffer.write(static_cast(command)); + if (!query.empty()) + { + buffer.write(query.data(), query.size()); + } + } }; } diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 979b5beef32..0a4474fdae0 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -141,6 +141,7 @@ enum ResponsePacketType PACKET_OK = 0x00, PACKET_ERR = 0xff, PACKET_EOF = 0xfe, + PACKET_LOCALINFILE = 0xfb, }; // https://dev.mysql.com/doc/dev/mysql-server/latest/group__group__cs__column__definition__flags.html @@ -422,6 +423,13 @@ public: uint64_t readLengthEncodedNumber(ReadBuffer & ss); +inline void readLengthEncodedString(String & s, ReadBuffer & buffer) +{ + uint64_t len = readLengthEncodedNumber(buffer); + s.resize(len); + buffer.readStrict(reinterpret_cast(s.data()), len); +} + void writeLengthEncodedNumber(uint64_t x, WriteBuffer & buffer); inline void writeLengthEncodedString(const String & s, WriteBuffer & buffer) @@ -453,7 +461,7 @@ public: String auth_plugin_name; String auth_plugin_data; - Handshake() = default; + Handshake() : connection_id(0x00), capability_flags(0x00), character_set(0x00), status_flags(0x00) { } Handshake(uint32_t capability_flags_, uint32_t connection_id_, String server_version_, String auth_plugin_name_, String auth_plugin_data_) : protocol_version(0xa) @@ -489,43 +497,43 @@ public: writeChar(0x0, 1, buffer); } - void readPayloadImpl(ReadBuffer & buffer) override + void readPayloadImpl(ReadBuffer & payload) override { - buffer.readStrict(reinterpret_cast(&protocol_version), 1); - readNullTerminated(server_version, buffer); - buffer.readStrict(reinterpret_cast(&connection_id), 4); + payload.readStrict(reinterpret_cast(&protocol_version), 1); + readNullTerminated(server_version, payload); + payload.readStrict(reinterpret_cast(&connection_id), 4); auth_plugin_data.resize(AUTH_PLUGIN_DATA_PART_1_LENGTH); - buffer.readStrict(auth_plugin_data.data(), AUTH_PLUGIN_DATA_PART_1_LENGTH); + payload.readStrict(auth_plugin_data.data(), AUTH_PLUGIN_DATA_PART_1_LENGTH); - buffer.ignore(1); - buffer.readStrict(reinterpret_cast(&capability_flags), 2); - buffer.readStrict(reinterpret_cast(&character_set), 1); - buffer.readStrict(reinterpret_cast(&status_flags), 2); - buffer.readStrict((reinterpret_cast(&capability_flags)) + 2, 2); + payload.ignore(1); + payload.readStrict(reinterpret_cast(&capability_flags), 2); + payload.readStrict(reinterpret_cast(&character_set), 1); + payload.readStrict(reinterpret_cast(&status_flags), 2); + payload.readStrict((reinterpret_cast(&capability_flags)) + 2, 2); UInt8 auth_plugin_data_length = 0; if (capability_flags & MySQLProtocol::CLIENT_PLUGIN_AUTH) { - buffer.readStrict(reinterpret_cast(&auth_plugin_data_length), 1); + payload.readStrict(reinterpret_cast(&auth_plugin_data_length), 1); } else { - buffer.ignore(1); + payload.ignore(1); } - buffer.ignore(10); + payload.ignore(10); if (capability_flags & MySQLProtocol::CLIENT_SECURE_CONNECTION) { UInt8 part2_length = (SCRAMBLE_LENGTH - AUTH_PLUGIN_DATA_PART_1_LENGTH); auth_plugin_data.resize(SCRAMBLE_LENGTH); - buffer.readStrict(auth_plugin_data.data() + AUTH_PLUGIN_DATA_PART_1_LENGTH, part2_length); - buffer.ignore(1); + payload.readStrict(auth_plugin_data.data() + AUTH_PLUGIN_DATA_PART_1_LENGTH, part2_length); + payload.ignore(1); } if (capability_flags & MySQLProtocol::CLIENT_PLUGIN_AUTH) { - readNullTerminated(auth_plugin_name, buffer); + readNullTerminated(auth_plugin_name, payload); } } }; @@ -649,9 +657,7 @@ public: if (capability_flags & CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA) { - auto len = readLengthEncodedNumber(payload); - auth_response.resize(len); - payload.readStrict(auth_response.data(), len); + readLengthEncodedString(auth_response, payload); } else if (capability_flags & CLIENT_SECURE_CONNECTION) { @@ -837,14 +843,10 @@ public: if (capabilities & CLIENT_SESSION_TRACK) { - auto len = readLengthEncodedNumber(payload); - info.resize(len); - payload.readStrict(info.data(), len); + readLengthEncodedString(info, payload); if (status_flags & SERVER_SESSION_STATE_CHANGED) { - len = readLengthEncodedNumber(payload); - session_state_changes.resize(len); - payload.readStrict(session_state_changes.data(), len); + readLengthEncodedString(session_state_changes, payload); } } else @@ -920,12 +922,16 @@ public: assert(header == 0xff); payload.readStrict(reinterpret_cast(&error_code), 2); - payload.ignore(1); - sql_state.resize(5); - payload.readStrict(reinterpret_cast(sql_state.data()), 5); - - readNullTerminated(error_message, payload); + /// SQL State [optional: # + 5bytes string] + UInt8 sharp = static_cast(*payload.position()); + if (sharp == 0x23) + { + payload.ignore(1); + sql_state.resize(5); + payload.readStrict(reinterpret_cast(sql_state.data()), 5); + } + readString(error_message, payload); } }; @@ -936,12 +942,13 @@ public: OK_Packet ok; ERR_Packet err; EOF_Packet eof; + UInt64 column_length = 0; PacketResponse(UInt32 server_capability_flags_) : ok(OK_Packet(server_capability_flags_)) { } void readPayloadImpl(ReadBuffer & payload) override { - UInt8 header = static_cast(*payload.position()); + UInt16 header = static_cast(*payload.position()); switch (header) { case PACKET_OK: @@ -956,6 +963,11 @@ public: packetType = PACKET_EOF; eof.readPayloadImpl(payload); break; + case PACKET_LOCALINFILE: + packetType = PACKET_LOCALINFILE; + break; + default: + column_length = readLengthEncodedNumber(payload); } } @@ -965,8 +977,9 @@ private: ResponsePacketType packetType = PACKET_OK; }; -class ColumnDefinition : public WritePacket +class ColumnDefinition : public WritePacket, public ReadPacket { +public: String schema; String table; String org_table; @@ -978,7 +991,9 @@ class ColumnDefinition : public WritePacket ColumnType column_type; uint16_t flags; uint8_t decimals = 0x00; -public: + + ColumnDefinition() : character_set(0x00), column_length(0), column_type(MYSQL_TYPE_DECIMAL), flags(0x00) { } + ColumnDefinition( String schema_, String table_, @@ -1009,7 +1024,6 @@ public: { } -protected: size_t getPayloadSize() const override { return 13 + getLengthEncodedStringSize("def") + getLengthEncodedStringSize(schema) + getLengthEncodedStringSize(table) + getLengthEncodedStringSize(org_table) + \ @@ -1032,6 +1046,25 @@ protected: buffer.write(reinterpret_cast(&decimals), 2); writeChar(0x0, 2, buffer); } + + void readPayloadImpl(ReadBuffer & payload) override + { + String def; + readLengthEncodedString(def, payload); + assert(def == "def"); + readLengthEncodedString(schema, payload); + readLengthEncodedString(table, payload); + readLengthEncodedString(org_table, payload); + readLengthEncodedString(name, payload); + readLengthEncodedString(org_name, payload); + next_length = readLengthEncodedNumber(payload); + payload.readStrict(reinterpret_cast(&character_set), 2); + payload.readStrict(reinterpret_cast(&column_length), 4); + payload.readStrict(reinterpret_cast(&column_type), 1); + payload.readStrict(reinterpret_cast(&flags), 2); + payload.readStrict(reinterpret_cast(&decimals), 2); + payload.ignore(2); + } }; class ComFieldList : public LimitedReadPacket diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index 870eb4966c6..68f9f68d8d3 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -11,15 +12,7 @@ int main(int, char **) using namespace MySQLProtocol; using namespace MySQLProtocol::Authentication; - /* - UInt16 port = 9001; - String host = "127.0.0.1", user = "default", password = "123"; - MySQLClient client(host, port, user, password, ""); - if (!client.connect()) - { - std::cerr << "Connect Error: " << client.error() << std::endl; - } - */ + String user = "default"; String password = "123"; String database = ""; @@ -90,9 +83,9 @@ int main(int, char **) server.writePayloadImpl(out0); // 2. Client reads packet - ReadBufferFromString in1(s0); + ReadBufferFromString in0(s0); PacketResponse client(server_capability_flags); - client.readPayloadImpl(in1); + client.readPayloadImpl(in0); // Check ASSERT(client.getType() == PACKET_OK); @@ -110,16 +103,16 @@ int main(int, char **) server.writePayloadImpl(out0); // 2. Client reads packet - ReadBufferFromString in1(s0); + ReadBufferFromString in0(s0); PacketResponse client(server_capability_flags); - client.readPayloadImpl(in1); + client.readPayloadImpl(in0); // Check ASSERT(client.getType() == PACKET_ERR); ASSERT(client.err.header == server.header); ASSERT(client.err.error_code == server.error_code); ASSERT(client.err.sql_state == server.sql_state); - ASSERT(client.err.error_message == server.error_message); + ASSERT(client.err.error_message.data() == server.error_message); } /// EOF Packet @@ -131,9 +124,9 @@ int main(int, char **) server.writePayloadImpl(out0); // 2. Client reads packet - ReadBufferFromString in1(s0); + ReadBufferFromString in0(s0); PacketResponse client(server_capability_flags); - client.readPayloadImpl(in1); + client.readPayloadImpl(in0); // Check ASSERT(client.getType() == PACKET_EOF); @@ -142,5 +135,57 @@ int main(int, char **) ASSERT(client.eof.status_flags == server.status_flags); } + /// ColumnDefinition Packet + { + // 1. Server writes packet + std::string s0; + WriteBufferFromString out0(s0); + ColumnDefinition server("schema", "tbl", "org_tbl", "name", "org_name", 33, 0x00, MYSQL_TYPE_STRING, 0x00, 0x00); + server.writePayloadImpl(out0); + + // 2. Client reads packet + ReadBufferFromString in0(s0); + ColumnDefinition client; + client.readPayloadImpl(in0); + + // Check + ASSERT(client.column_type == server.column_type); + ASSERT(client.column_length == server.column_length); + ASSERT(client.next_length == server.next_length); + ASSERT(client.character_set == server.character_set); + ASSERT(client.decimals == server.decimals); + ASSERT(client.name == server.name); + ASSERT(client.org_name == server.org_name); + ASSERT(client.table == server.table); + ASSERT(client.org_table == server.org_table); + ASSERT(client.schema == server.schema); + } + + { + MySQLClient client1("127.0.0.1", 9001, "default", "123", ""); + if (!client1.connect()) + { + std::cerr << "Connect Error: " << client1.error() << std::endl; + return 1; + } + + if (!client1.ping()) + { + std::cerr << "Connect Error: " << client1.error() << std::endl; + return 1; + } + + if (!client1.initdb("default")) + { + std::cerr << "Connect Error: " << client1.error() << std::endl; + return 1; + } + + if (!client1.query("select 1")) + { + std::cerr << "Connect Error: " << client1.error() << std::endl; + return 1; + } + } return 0; } From 44cd5534b4c5303db9a2ea0056b218e8f2a81d1b Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Thu, 23 Apr 2020 17:27:53 +0800 Subject: [PATCH 030/374] make getPayloadSize protected --- src/Core/MySQLProtocol.h | 188 +++++++++++++++++++++------------------ 1 file changed, 100 insertions(+), 88 deletions(-) diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 0a4474fdae0..237929ed068 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -475,11 +475,6 @@ public: { } - size_t getPayloadSize() const override - { - return 26 + server_version.size() + auth_plugin_data.size() + auth_plugin_name.size(); - } - void writePayloadImpl(WriteBuffer & buffer) const override { buffer.write(static_cast(protocol_version)); @@ -536,6 +531,12 @@ public: readNullTerminated(auth_plugin_name, payload); } } + +protected: + size_t getPayloadSize() const override + { + return 26 + server_version.size() + auth_plugin_data.size() + auth_plugin_name.size(); + } }; class SSLRequest : public ReadPacket @@ -556,15 +557,15 @@ public: class HandshakeResponse : public WritePacket, public ReadPacket { public: - uint32_t capability_flags = 0; - uint32_t max_packet_size = 0; - uint8_t character_set = 0; + uint32_t capability_flags; + uint32_t max_packet_size; + uint8_t character_set; String username; String database; String auth_response; String auth_plugin_name; - HandshakeResponse() = default; + HandshakeResponse() : capability_flags(0x00), max_packet_size(0x00), character_set(0x00) { } HandshakeResponse( UInt32 capability_flags_, @@ -584,35 +585,6 @@ public: { } - size_t getPayloadSize() const override - { - size_t size = 0; - size += 4 + 4 + 1 + 23; - size += username.size() + 1; - - if (capability_flags & CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA) - { - size += getLengthEncodedStringSize(auth_response); - } - else if (capability_flags & CLIENT_SECURE_CONNECTION) - { - size += (1 + auth_response.size()); - } - else - { - size += (auth_response.size() + 1); - } - if (capability_flags & CLIENT_CONNECT_WITH_DB) - { - size += (database.size() + 1); - } - if (capability_flags & CLIENT_PLUGIN_AUTH) - { - size += (auth_plugin_name.size() + 1); - } - return size; - } - void writePayloadImpl(WriteBuffer & buffer) const override { buffer.write(reinterpret_cast(&capability_flags), 4); @@ -681,6 +653,36 @@ public: readNullTerminated(auth_plugin_name, payload); } } + +protected: + size_t getPayloadSize() const override + { + size_t size = 0; + size += 4 + 4 + 1 + 23; + size += username.size() + 1; + + if (capability_flags & CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA) + { + size += getLengthEncodedStringSize(auth_response); + } + else if (capability_flags & CLIENT_SECURE_CONNECTION) + { + size += (1 + auth_response.size()); + } + else + { + size += (auth_response.size() + 1); + } + if (capability_flags & CLIENT_CONNECT_WITH_DB) + { + size += (database.size() + 1); + } + if (capability_flags & CLIENT_PLUGIN_AUTH) + { + size += (auth_plugin_name.size() + 1); + } + return size; + } }; class AuthSwitchRequest : public WritePacket @@ -750,7 +752,11 @@ public: String session_state_changes; String info; - OK_Packet(uint32_t capabilities_) : header(0x00), capabilities(capabilities_), affected_rows(0), last_insert_id(0), status_flags(0) { } + OK_Packet(uint32_t capabilities_) + : header(0x00), capabilities(capabilities_), affected_rows(0x00), last_insert_id(0x00), status_flags(0x00) + { + } + OK_Packet( uint8_t header_, uint32_t capabilities_, @@ -770,33 +776,6 @@ public: { } - size_t getPayloadSize() const override - { - size_t result = 2 + getLengthEncodedNumberSize(affected_rows); - - if (capabilities & CLIENT_PROTOCOL_41) - { - result += 4; - } - else if (capabilities & CLIENT_TRANSACTIONS) - { - result += 2; - } - - if (capabilities & CLIENT_SESSION_TRACK) - { - result += getLengthEncodedStringSize(info); - if (status_flags & SERVER_SESSION_STATE_CHANGED) - result += getLengthEncodedStringSize(session_state_changes); - } - else - { - result += info.size(); - } - - return result; - } - void writePayloadImpl(WriteBuffer & buffer) const override { buffer.write(header); @@ -854,23 +833,46 @@ public: readString(info, payload); } } + +protected: + size_t getPayloadSize() const override + { + size_t result = 2 + getLengthEncodedNumberSize(affected_rows); + + if (capabilities & CLIENT_PROTOCOL_41) + { + result += 4; + } + else if (capabilities & CLIENT_TRANSACTIONS) + { + result += 2; + } + + if (capabilities & CLIENT_SESSION_TRACK) + { + result += getLengthEncodedStringSize(info); + if (status_flags & SERVER_SESSION_STATE_CHANGED) + result += getLengthEncodedStringSize(session_state_changes); + } + else + { + result += info.size(); + } + + return result; + } }; class EOF_Packet : public WritePacket, public ReadPacket { public: UInt8 header = 0xfe; - int warnings = 0; - int status_flags = 0; + int warnings; + int status_flags; - EOF_Packet() = default; - EOF_Packet(int warnings_, int status_flags_) : warnings(warnings_), status_flags(status_flags_) - {} + EOF_Packet() : warnings(0x00), status_flags(0x00) { } - size_t getPayloadSize() const override - { - return 5; - } + EOF_Packet(int warnings_, int status_flags_) : warnings(warnings_), status_flags(status_flags_) { } void writePayloadImpl(WriteBuffer & buffer) const override { @@ -886,27 +888,29 @@ public: payload.readStrict(reinterpret_cast(&warnings), 2); payload.readStrict(reinterpret_cast(&status_flags), 2); } + +protected: + size_t getPayloadSize() const override + { + return 5; + } }; class ERR_Packet : public WritePacket, public ReadPacket { public: UInt8 header = 0xff; - int error_code = 0; + int error_code; String sql_state; String error_message; - ERR_Packet() = default; + ERR_Packet() : error_code(0x00) { } + ERR_Packet(int error_code_, String sql_state_, String error_message_) : error_code(error_code_), sql_state(std::move(sql_state_)), error_message(std::move(error_message_)) { } - size_t getPayloadSize() const override - { - return 4 + sql_state.length() + std::min(error_message.length(), MYSQL_ERRMSG_SIZE); - } - void writePayloadImpl(WriteBuffer & buffer) const override { buffer.write(header); @@ -933,6 +937,12 @@ public: } readString(error_message, payload); } + +protected: + size_t getPayloadSize() const override + { + return 4 + sql_state.length() + std::min(error_message.length(), MYSQL_ERRMSG_SIZE); + } }; /// https://dev.mysql.com/doc/internals/en/generic-response-packets.html @@ -967,6 +977,7 @@ public: packetType = PACKET_LOCALINFILE; break; default: + packetType = PACKET_OK; column_length = readLengthEncodedNumber(payload); } } @@ -1024,12 +1035,6 @@ public: { } - size_t getPayloadSize() const override - { - return 13 + getLengthEncodedStringSize("def") + getLengthEncodedStringSize(schema) + getLengthEncodedStringSize(table) + getLengthEncodedStringSize(org_table) + \ - getLengthEncodedStringSize(name) + getLengthEncodedStringSize(org_name) + getLengthEncodedNumberSize(next_length); - } - void writePayloadImpl(WriteBuffer & buffer) const override { writeLengthEncodedString(std::string("def"), buffer); /// always "def" @@ -1065,6 +1070,13 @@ public: payload.readStrict(reinterpret_cast(&decimals), 2); payload.ignore(2); } + +protected: + size_t getPayloadSize() const override + { + return 13 + getLengthEncodedStringSize("def") + getLengthEncodedStringSize(schema) + getLengthEncodedStringSize(table) + getLengthEncodedStringSize(org_table) + \ + getLengthEncodedStringSize(name) + getLengthEncodedStringSize(org_name) + getLengthEncodedNumberSize(next_length); + } }; class ComFieldList : public LimitedReadPacket From 4a21fce889dd55e077c074ac49196606038a7b77 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Thu, 23 Apr 2020 18:58:11 +0800 Subject: [PATCH 031/374] fix packet response capability to client --- src/Core/MySQLClient.cpp | 5 ++--- src/Core/MySQLClient.h | 1 - 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index ad7f7b66c63..4ea75206ddb 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -54,7 +54,6 @@ bool MySQLClient::handshake() "Only support " + mysql_native_password + " auth plugin name, but got " + handshake.auth_plugin_name, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); } - server_capability_flags = handshake.capability_flags; Native41 native41(password, handshake.auth_plugin_data); String auth_plugin_data = native41.getAuthPluginData(); @@ -63,7 +62,7 @@ bool MySQLClient::handshake() client_capability_flags, max_packet_size, charset_utf8, user, database, auth_plugin_data, mysql_native_password); packet_sender->sendPacket(handshake_response, true); - PacketResponse packet_response(server_capability_flags); + PacketResponse packet_response(client_capability_flags); packet_sender->receivePacket(packet_response); packet_sender->resetSequenceId(); if (packet_response.getType() == PACKET_ERR) @@ -100,7 +99,7 @@ bool MySQLClient::writeCommand(char command, String query) WriteCommand write_command(command, query); packet_sender->sendPacket(write_command, true); - PacketResponse packet_response(server_capability_flags); + PacketResponse packet_response(client_capability_flags); packet_sender->receivePacket(packet_response); switch (packet_response.getType()) { diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 2140f1032a7..43edfbe963d 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -37,7 +37,6 @@ private: bool connected = false; String last_error; UInt32 client_capability_flags = 0; - UInt32 server_capability_flags = 0; uint8_t seq = 0; UInt8 charset_utf8 = 33; From 27e2fe2778b809e117682f130fb36fa704dbcedf Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Thu, 23 Apr 2020 22:08:01 +0800 Subject: [PATCH 032/374] add register slave command --- src/Core/MySQLClient.cpp | 34 +++++++++++++----------- src/Core/MySQLClient.h | 3 +-- src/Core/MySQLProtocol.h | 44 +++++++++++++++++++++++++++++++ src/Core/tests/mysql_protocol.cpp | 8 +----- 4 files changed, 64 insertions(+), 25 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 4ea75206ddb..76421172c42 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -2,6 +2,7 @@ namespace DB { +using namespace ReplicationProtocol; using namespace MySQLProtocol::Authentication; namespace ErrorCodes @@ -72,6 +73,22 @@ bool MySQLClient::handshake() return (packet_response.getType() != PACKET_ERR); } + +bool MySQLClient::register_slave(UInt32 server_id) +{ + RegisterSlave register_slave(server_id); + packet_sender->sendPacket(register_slave, true); + + PacketResponse packet_response(client_capability_flags); + packet_sender->receivePacket(packet_response); + packet_sender->resetSequenceId(); + if (packet_response.getType() == PACKET_ERR) + { + last_error = packet_response.err.error_message; + } + return (packet_response.getType() != PACKET_ERR); +} + bool MySQLClient::ping() { return writeCommand(Command::COM_PING, ""); @@ -82,11 +99,6 @@ bool MySQLClient::initdb(String db) return writeCommand(Command::COM_INIT_DB, db); } -bool MySQLClient::query(String q) -{ - return writeCommand(Command::COM_QUERY, q); -} - String MySQLClient::error() { return last_error; @@ -107,7 +119,7 @@ bool MySQLClient::writeCommand(char command, String query) last_error = packet_response.err.error_message; break; case PACKET_OK: - ret = readColumns(packet_response.column_length); + ret = true; break; default: break; @@ -115,14 +127,4 @@ bool MySQLClient::writeCommand(char command, String query) packet_sender->resetSequenceId(); return ret; } - -bool MySQLClient::readColumns(int column_length) -{ - for (auto i = 0; i < column_length; i++) - { - ColumnDefinition cd; - packet_sender->receivePacket(cd); - } - return true; -} } diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 43edfbe963d..16b4b01c4ae 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -24,7 +24,7 @@ public: void disconnect(); bool ping(); bool initdb(String db); - bool query(String q); + bool register_slave(UInt32 server_id); String error(); private: @@ -50,7 +50,6 @@ private: std::shared_ptr packet_sender; bool handshake(); - bool readColumns(int column_length); bool writeCommand(char command, String query); }; diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 237929ed068..982273f6c14 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -102,6 +102,7 @@ enum Command COM_TIME = 0xf, COM_DELAYED_INSERT = 0x10, COM_CHANGE_USER = 0x11, + COM_REGISTER_SLAVE = 0x15, COM_RESET_CONNECTION = 0x1f, COM_DAEMON = 0x1d }; @@ -1434,5 +1435,48 @@ private: } +namespace ReplicationProtocol +{ + /// https://dev.mysql.com/doc/internals/en/com-register-slave.html + class RegisterSlave : public WritePacket + { + public: + UInt8 header = COM_REGISTER_SLAVE; + UInt32 server_id; + String slaves_hostname; + String slaves_users; + String slaves_password; + size_t slaves_mysql_port; + UInt32 replication_rank; + UInt32 master_id; + + RegisterSlave(UInt32 server_id_) + : server_id(server_id_) + , slaves_mysql_port(0x00) + , replication_rank(0x00) + , master_id(0x00) + { + } + + void writePayloadImpl(WriteBuffer & buffer) const override + { + buffer.write(header); + buffer.write(reinterpret_cast(&server_id), 4); + writeLengthEncodedString(slaves_hostname, buffer); + writeLengthEncodedString(slaves_users, buffer); + writeLengthEncodedString(slaves_password, buffer); + buffer.write(reinterpret_cast(&slaves_mysql_port), 2); + buffer.write(reinterpret_cast(&replication_rank), 4); + buffer.write(reinterpret_cast(&master_id), 4); + } + + protected: + size_t getPayloadSize() const override + { + return 1 + 4 + getLengthEncodedStringSize(slaves_hostname) + getLengthEncodedStringSize(slaves_users) + + getLengthEncodedStringSize(slaves_password) + 2 + 4 + 4; + } + }; +} } } diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index 68f9f68d8d3..fd4c12d68c8 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -175,13 +175,7 @@ int main(int, char **) return 1; } - if (!client1.initdb("default")) - { - std::cerr << "Connect Error: " << client1.error() << std::endl; - return 1; - } - - if (!client1.query("select 1")) + if (!client1.register_slave(123)) { std::cerr << "Connect Error: " << client1.error() << std::endl; return 1; From ef4126d533f4a1456b565cd239419e2779adfd4f Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Sat, 25 Apr 2020 12:19:32 +0800 Subject: [PATCH 033/374] add com_binlog_dump --- src/Core/MySQLClient.cpp | 27 ++++++++++++++++++--------- src/Core/MySQLClient.h | 4 ++-- src/Core/MySQLProtocol.h | 30 ++++++++++++++++++++++++++++++ src/Core/tests/mysql_protocol.cpp | 5 +++-- 4 files changed, 53 insertions(+), 13 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 76421172c42..056e5d8d303 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -73,10 +73,14 @@ bool MySQLClient::handshake() return (packet_response.getType() != PACKET_ERR); } - -bool MySQLClient::register_slave(UInt32 server_id) +bool MySQLClient::ping() { - RegisterSlave register_slave(server_id); + return writeCommand(Command::COM_PING, ""); +} + +bool MySQLClient::register_slave(UInt32 slave_id) +{ + RegisterSlave register_slave(slave_id); packet_sender->sendPacket(register_slave, true); PacketResponse packet_response(client_capability_flags); @@ -89,14 +93,19 @@ bool MySQLClient::register_slave(UInt32 server_id) return (packet_response.getType() != PACKET_ERR); } -bool MySQLClient::ping() +bool MySQLClient::binlog_dump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos) { - return writeCommand(Command::COM_PING, ""); -} + BinlogDump binlog_dump(binlog_pos, binlog_file_name, slave_id); + packet_sender->sendPacket(binlog_dump, true); -bool MySQLClient::initdb(String db) -{ - return writeCommand(Command::COM_INIT_DB, db); + PacketResponse packet_response(client_capability_flags); + packet_sender->receivePacket(packet_response); + packet_sender->resetSequenceId(); + if (packet_response.getType() == PACKET_ERR) + { + last_error = packet_response.err.error_message; + } + return (packet_response.getType() != PACKET_ERR); } String MySQLClient::error() diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 16b4b01c4ae..0c55165ac7d 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -23,8 +23,8 @@ public: bool connect(); void disconnect(); bool ping(); - bool initdb(String db); - bool register_slave(UInt32 server_id); + bool register_slave(UInt32 slave_id); + bool binlog_dump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos); String error(); private: diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 982273f6c14..3f2c490e6c6 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -102,6 +102,7 @@ enum Command COM_TIME = 0xf, COM_DELAYED_INSERT = 0x10, COM_CHANGE_USER = 0x11, + COM_BINLOG_DUMP = 0x12, COM_REGISTER_SLAVE = 0x15, COM_RESET_CONNECTION = 0x1f, COM_DAEMON = 0x1d @@ -1477,6 +1478,35 @@ namespace ReplicationProtocol + getLengthEncodedStringSize(slaves_password) + 2 + 4 + 4; } }; + + /// https://dev.mysql.com/doc/internals/en/com-binlog-dump.html + class BinlogDump : public WritePacket + { + public: + UInt8 header = COM_BINLOG_DUMP; + UInt32 binlog_pos; + UInt16 flags; + UInt32 server_id; + String binlog_file_name; + + BinlogDump(UInt32 binlog_pos_, String binlog_file_name_, UInt32 server_id_) + : binlog_pos(binlog_pos_), flags(0x00), server_id(server_id_), binlog_file_name(std::move(binlog_file_name_)) + { + } + + void writePayloadImpl(WriteBuffer & buffer) const override + { + buffer.write(header); + buffer.write(reinterpret_cast(&binlog_pos), 4); + buffer.write(reinterpret_cast(&flags), 2); + buffer.write(reinterpret_cast(&server_id), 4); + buffer.write(binlog_file_name.data(), binlog_file_name.length()); + buffer.write(0x00); + } + + protected: + size_t getPayloadSize() const override { return 1 + 4 + 2 + 4 + binlog_file_name.size() + 1; } + }; } } } diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index fd4c12d68c8..2777025c2f4 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -162,6 +162,7 @@ int main(int, char **) } { + UInt32 slave_id = 123; MySQLClient client1("127.0.0.1", 9001, "default", "123", ""); if (!client1.connect()) { @@ -175,10 +176,10 @@ int main(int, char **) return 1; } - if (!client1.register_slave(123)) + if (!client1.binlog_dump(slave_id, "", 0)) { std::cerr << "Connect Error: " << client1.error() << std::endl; - return 1; + assert(0); } } return 0; From 99402fac85a7549d31b9a43201a3b4f1d6e96bb1 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Sat, 25 Apr 2020 12:56:47 +0800 Subject: [PATCH 034/374] fixup clang tidy errors --- src/Core/MySQLClient.cpp | 80 +++++++++++++++---------------- src/Core/MySQLClient.h | 4 +- src/Core/tests/mysql_protocol.cpp | 2 +- 3 files changed, 43 insertions(+), 43 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 056e5d8d303..c538c3d0759 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -73,46 +73,6 @@ bool MySQLClient::handshake() return (packet_response.getType() != PACKET_ERR); } -bool MySQLClient::ping() -{ - return writeCommand(Command::COM_PING, ""); -} - -bool MySQLClient::register_slave(UInt32 slave_id) -{ - RegisterSlave register_slave(slave_id); - packet_sender->sendPacket(register_slave, true); - - PacketResponse packet_response(client_capability_flags); - packet_sender->receivePacket(packet_response); - packet_sender->resetSequenceId(); - if (packet_response.getType() == PACKET_ERR) - { - last_error = packet_response.err.error_message; - } - return (packet_response.getType() != PACKET_ERR); -} - -bool MySQLClient::binlog_dump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos) -{ - BinlogDump binlog_dump(binlog_pos, binlog_file_name, slave_id); - packet_sender->sendPacket(binlog_dump, true); - - PacketResponse packet_response(client_capability_flags); - packet_sender->receivePacket(packet_response); - packet_sender->resetSequenceId(); - if (packet_response.getType() == PACKET_ERR) - { - last_error = packet_response.err.error_message; - } - return (packet_response.getType() != PACKET_ERR); -} - -String MySQLClient::error() -{ - return last_error; -} - bool MySQLClient::writeCommand(char command, String query) { bool ret = false; @@ -136,4 +96,44 @@ bool MySQLClient::writeCommand(char command, String query) packet_sender->resetSequenceId(); return ret; } + +bool MySQLClient::ping() +{ + return writeCommand(Command::COM_PING, ""); +} + +bool MySQLClient::registerSlave(UInt32 slave_id) +{ + RegisterSlave register_slave(slave_id); + packet_sender->sendPacket(register_slave, true); + + PacketResponse packet_response(client_capability_flags); + packet_sender->receivePacket(packet_response); + packet_sender->resetSequenceId(); + if (packet_response.getType() == PACKET_ERR) + { + last_error = packet_response.err.error_message; + } + return (packet_response.getType() != PACKET_ERR); +} + +bool MySQLClient::binlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos) +{ + BinlogDump binlog_dump(binlog_pos, binlog_file_name, slave_id); + packet_sender->sendPacket(binlog_dump, true); + + PacketResponse packet_response(client_capability_flags); + packet_sender->receivePacket(packet_response); + packet_sender->resetSequenceId(); + if (packet_response.getType() == PACKET_ERR) + { + last_error = packet_response.err.error_message; + } + return (packet_response.getType() != PACKET_ERR); +} + +String MySQLClient::error() +{ + return last_error; +} } diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 0c55165ac7d..12ebbca45f4 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -23,8 +23,8 @@ public: bool connect(); void disconnect(); bool ping(); - bool register_slave(UInt32 slave_id); - bool binlog_dump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos); + bool registerSlave(UInt32 slave_id); + bool binlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos); String error(); private: diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index 2777025c2f4..46cee9709f5 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -176,7 +176,7 @@ int main(int, char **) return 1; } - if (!client1.binlog_dump(slave_id, "", 0)) + if (!client1.binlogDump(slave_id, "", 0)) { std::cerr << "Connect Error: " << client1.error() << std::endl; assert(0); From 0d3cab3e10550144a27ec86ee06c3309e3261659 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Sat, 25 Apr 2020 22:04:36 +0800 Subject: [PATCH 035/374] fix openssl build --- src/Core/MySQLClient.h | 2 + src/Core/MySQLProtocol.h | 70 +++++++++++++++++++++++++++---- src/Core/tests/CMakeLists.txt | 3 ++ src/Core/tests/mysql_protocol.cpp | 2 +- 4 files changed, 68 insertions(+), 9 deletions(-) diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 12ebbca45f4..96eb12cccba 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -15,6 +15,7 @@ namespace DB { using namespace MySQLProtocol; +using namespace ReplicationProtocol; class MySQLClient { @@ -25,6 +26,7 @@ public: bool ping(); bool registerSlave(UInt32 slave_id); bool binlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos); + BinlogEvent readBinlogEvent(); String error(); private: diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 3f2c490e6c6..8789c43b2fd 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -1451,13 +1451,7 @@ namespace ReplicationProtocol UInt32 replication_rank; UInt32 master_id; - RegisterSlave(UInt32 server_id_) - : server_id(server_id_) - , slaves_mysql_port(0x00) - , replication_rank(0x00) - , master_id(0x00) - { - } + RegisterSlave(UInt32 server_id_) : server_id(server_id_), slaves_mysql_port(0x00), replication_rank(0x00), master_id(0x00) { } void writePayloadImpl(WriteBuffer & buffer) const override { @@ -1507,6 +1501,66 @@ namespace ReplicationProtocol protected: size_t getPayloadSize() const override { return 1 + 4 + 2 + 4 + binlog_file_name.size() + 1; } }; -} + + class BinlogEvent + { + public: + BinlogEvent() = default; + + virtual bool isValid(); + + virtual void parseEvents(ReadBuffer & payload); + + virtual ~BinlogEvent(); + }; + + class MySQLBinlogEvent : public BinlogEvent + { + public: + MySQLBinlogEvent() = default; + + void parseEvents(ReadBuffer & payload) override + { + UInt16 header = 0; + payload.readStrict(reinterpret_cast(&header), 1); + } + }; + + class IFlavor + { + public: + virtual String getName() = 0; + + virtual BinlogEvent readBinlogEvent(ReadBuffer & payload); + + virtual ~IFlavor() = default; + }; + + class MySQLFlavor : public IFlavor + { + public: + MySQLFlavor() = default; + + BinlogEvent readBinlogEvent(ReadBuffer & payload) override + { + UInt16 header = 0; + payload.readStrict(reinterpret_cast(&header), 1); + switch (header) + { + case PACKET_EOF: + throw Exception("Master maybe lost", ErrorCodes::UNKNOWN_EXCEPTION); + case PACKET_ERR: + ERR_Packet err; + err.readPayloadImpl(payload); + throw Exception(err.error_message, ErrorCodes::UNKNOWN_EXCEPTION); + } + + MySQLBinlogEvent binlog_event; + binlog_event.parseEvents(payload); + return binlog_event; + } + }; +} + } } diff --git a/src/Core/tests/CMakeLists.txt b/src/Core/tests/CMakeLists.txt index a5d694b7358..f77163a7109 100644 --- a/src/Core/tests/CMakeLists.txt +++ b/src/Core/tests/CMakeLists.txt @@ -21,3 +21,6 @@ target_link_libraries (mysql_client PRIVATE dbms) add_executable (mysql_protocol mysql_protocol.cpp) target_link_libraries (mysql_protocol PRIVATE dbms) +if(USE_SSL) + target_include_directories (mysql_protocol SYSTEM PRIVATE ${OPENSSL_INCLUDE_DIR}) +endif() diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index 46cee9709f5..a6aeb4d16ea 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -176,7 +176,7 @@ int main(int, char **) return 1; } - if (!client1.binlogDump(slave_id, "", 0)) + if (!client1.binlogDump(slave_id, "", 315)) { std::cerr << "Connect Error: " << client1.error() << std::endl; assert(0); From c91a810d2d2100fb56b5736bb890891ad121d661 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 27 Apr 2020 23:01:46 +0800 Subject: [PATCH 036/374] add binlog event --- src/Core/MySQLClient.cpp | 4 +- src/Core/MySQLClient.h | 2 - src/Core/MySQLProtocol.h | 121 ++++++++++++++++++++++++++++++++------- 3 files changed, 103 insertions(+), 24 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index c538c3d0759..457da104491 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -2,8 +2,8 @@ namespace DB { -using namespace ReplicationProtocol; -using namespace MySQLProtocol::Authentication; +using namespace Replication; +using namespace Authentication; namespace ErrorCodes { diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 96eb12cccba..12ebbca45f4 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -15,7 +15,6 @@ namespace DB { using namespace MySQLProtocol; -using namespace ReplicationProtocol; class MySQLClient { @@ -26,7 +25,6 @@ public: bool ping(); bool registerSlave(UInt32 slave_id); bool binlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos); - BinlogEvent readBinlogEvent(); String error(); private: diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 8789c43b2fd..db24342846e 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -1436,7 +1436,7 @@ private: } -namespace ReplicationProtocol +namespace Replication { /// https://dev.mysql.com/doc/internals/en/com-register-slave.html class RegisterSlave : public WritePacket @@ -1501,29 +1501,106 @@ namespace ReplicationProtocol protected: size_t getPayloadSize() const override { return 1 + 4 + 2 + 4 + binlog_file_name.size() + 1; } }; +} +} - class BinlogEvent +namespace BinlogEvent +{ + using namespace MySQLProtocol; + + class IBinlogEvent; + using BinlogEventPtr = std::shared_ptr; + + /// http://dev.mysql.com/doc/internals/en/binlog-event-type.html + enum EventType + { + UNKNOW_EVENT = 0, + START_EVENT_V3 = 1, + QUERY_EVENT = 2, + STOP_EVENT = 3, + ROTATE_EVENT = 4, + INT_VAR_EVENT = 5, + LOAD_EVENT = 6, + SLAVE_EVENT = 7, + CREATE_FILE_EVENT = 8, + APPEND_BLOCK_EVENT = 9, + EXEC_LOAD_EVENT = 10, + DELETE_FILE_EVENT = 11, + NEW_LOAD_EVENT = 12, + RAND_EVENT = 13, + USER_VAR_EVENT = 14, + FORMAT_DESCRIPTION_EVENT = 15, + XID_EVENT = 16, + BEGIN_LOAD_QUERY_EVENT = 17, + EXECUTE_LOAD_QUERY_EVENT = 18, + TABLE_MAP_EVENT = 19, + WRITE_ROWS_EVENT_V0 = 20, + UPDATE_ROWS_EVENT_V0 = 21, + DELETE_ROWS_EVENT_V0 = 22, + WRITE_ROWS_EVENT_V1 = 23, + UPDATE_ROWS_EVENT_V1 = 24, + DELETE_ROWS_EVENT_V1 = 25, + INCIDENT_EVENT = 26, + HEARTBEAT_EVENT = 27, + IGNORABLE_EVENT = 28, + ROWS_QUERY_EVENT = 29, + WRITE_ROWS_EVENT_V2 = 30, + UPDATE_ROWS_EVENT_V2 = 31, + DELETE_ROWS_EVENT_V2 = 32, + GTID_EVENT = 33, + ANONYMOUS_GTID_EVENT = 34, + PREVIOUS_GTIDS_EVENT = 35, + TRANSACTION_CONTEXT_EVENT = 36, + VIEW_CHANGE_EVENT = 37, + XA_PREPARE_LOG_EVENT = 38, + + /// MariaDB specific values. They start at 160. + MARIA_ANNOTATE_ROWS_EVENT = 160, + MARIA_BINLOG_CHECKPOINT_EVENT = 161, + MARIA_GTID_EVENT = 162, + MARIA_GTID_LIST_EVENT = 163, + MARIA_START_ENCRYPTION_EVENT = 164, + }; + + class EventHeader { public: - BinlogEvent() = default; + UInt32 timestamp; + EventType type; + UInt32 server_id; + UInt32 event_size; + UInt32 log_pos; + UInt16 flags; + + EventHeader() = default; + + void parse(ReadBuffer & payload) + { + payload.readStrict(reinterpret_cast(×tamp), 4); + payload.readStrict(reinterpret_cast(&type), 1); + payload.readStrict(reinterpret_cast(&server_id), 4); + payload.readStrict(reinterpret_cast(&event_size), 4); + payload.readStrict(reinterpret_cast(&log_pos), 4); + payload.readStrict(reinterpret_cast(&flags), 2); + } + }; + + class IBinlogEvent + { + public: + IBinlogEvent() = default; virtual bool isValid(); - virtual void parseEvents(ReadBuffer & payload); + virtual void parse(ReadBuffer & payload); - virtual ~BinlogEvent(); + virtual ~IBinlogEvent(); }; - class MySQLBinlogEvent : public BinlogEvent + class SampleEvent : public IBinlogEvent { public: - MySQLBinlogEvent() = default; - - void parseEvents(ReadBuffer & payload) override - { - UInt16 header = 0; - payload.readStrict(reinterpret_cast(&header), 1); - } + SampleEvent() = default; }; class IFlavor @@ -1531,7 +1608,7 @@ namespace ReplicationProtocol public: virtual String getName() = 0; - virtual BinlogEvent readBinlogEvent(ReadBuffer & payload); + virtual BinlogEventPtr readBinlogEvent(ReadBuffer & payload); virtual ~IFlavor() = default; }; @@ -1541,7 +1618,7 @@ namespace ReplicationProtocol public: MySQLFlavor() = default; - BinlogEvent readBinlogEvent(ReadBuffer & payload) override + BinlogEventPtr readBinlogEvent(ReadBuffer & payload) override { UInt16 header = 0; payload.readStrict(reinterpret_cast(&header), 1); @@ -1555,12 +1632,16 @@ namespace ReplicationProtocol throw Exception(err.error_message, ErrorCodes::UNKNOWN_EXCEPTION); } - MySQLBinlogEvent binlog_event; - binlog_event.parseEvents(payload); - return binlog_event; + EventHeader event_header; + event_header.parse(payload); + + switch (event_header.type) + { + default: + break; + } + return nullptr; } }; } - -} } From e32fc1865d2facbe13ccd9952f71ff8b88531944 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 28 Apr 2020 08:48:13 +0800 Subject: [PATCH 037/374] fix clang build --- src/Core/tests/mysql_protocol.cpp | 70 +++++++++++++++---------------- 1 file changed, 35 insertions(+), 35 deletions(-) diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index a6aeb4d16ea..8f6d9dfa7a9 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -42,12 +42,12 @@ int main(int, char **) client_handshake.readPayloadImpl(in0); /// Check packet - ASSERT(server_handshake.capability_flags == client_handshake.capability_flags); - ASSERT(server_handshake.status_flags == client_handshake.status_flags); - ASSERT(server_handshake.server_version == client_handshake.server_version); - ASSERT(server_handshake.protocol_version == client_handshake.protocol_version); - ASSERT(server_handshake.auth_plugin_data.substr(0, 20) == client_handshake.auth_plugin_data); - ASSERT(server_handshake.auth_plugin_name == client_handshake.auth_plugin_name); + ASSERT(server_handshake.capability_flags == client_handshake.capability_flags) + ASSERT(server_handshake.status_flags == client_handshake.status_flags) + ASSERT(server_handshake.server_version == client_handshake.server_version) + ASSERT(server_handshake.protocol_version == client_handshake.protocol_version) + ASSERT(server_handshake.auth_plugin_data.substr(0, 20) == client_handshake.auth_plugin_data) + ASSERT(server_handshake.auth_plugin_name == client_handshake.auth_plugin_name) /// 2. Greeting Response: std::string s1; @@ -66,12 +66,12 @@ int main(int, char **) server_handshake_response.readPayloadImpl(in1); /// Check - ASSERT(server_handshake_response.capability_flags == client_handshake_response.capability_flags); - ASSERT(server_handshake_response.character_set == client_handshake_response.character_set); - ASSERT(server_handshake_response.username == client_handshake_response.username); - ASSERT(server_handshake_response.database == client_handshake_response.database); - ASSERT(server_handshake_response.auth_response == client_handshake_response.auth_response); - ASSERT(server_handshake_response.auth_plugin_name == client_handshake_response.auth_plugin_name); + ASSERT(server_handshake_response.capability_flags == client_handshake_response.capability_flags) + ASSERT(server_handshake_response.character_set == client_handshake_response.character_set) + ASSERT(server_handshake_response.username == client_handshake_response.username) + ASSERT(server_handshake_response.database == client_handshake_response.database) + ASSERT(server_handshake_response.auth_response == client_handshake_response.auth_response) + ASSERT(server_handshake_response.auth_plugin_name == client_handshake_response.auth_plugin_name) } /// OK Packet @@ -88,10 +88,10 @@ int main(int, char **) client.readPayloadImpl(in0); // Check - ASSERT(client.getType() == PACKET_OK); - ASSERT(client.ok.header == server.header); - ASSERT(client.ok.status_flags == server.status_flags); - ASSERT(client.ok.capabilities == server.capabilities); + ASSERT(client.getType() == PACKET_OK) + ASSERT(client.ok.header == server.header) + ASSERT(client.ok.status_flags == server.status_flags) + ASSERT(client.ok.capabilities == server.capabilities) } /// ERR Packet @@ -108,11 +108,11 @@ int main(int, char **) client.readPayloadImpl(in0); // Check - ASSERT(client.getType() == PACKET_ERR); - ASSERT(client.err.header == server.header); - ASSERT(client.err.error_code == server.error_code); - ASSERT(client.err.sql_state == server.sql_state); - ASSERT(client.err.error_message.data() == server.error_message); + ASSERT(client.getType() == PACKET_ERR) + ASSERT(client.err.header == server.header) + ASSERT(client.err.error_code == server.error_code) + ASSERT(client.err.sql_state == server.sql_state) + ASSERT(client.err.error_message.data() == server.error_message) } /// EOF Packet @@ -129,10 +129,10 @@ int main(int, char **) client.readPayloadImpl(in0); // Check - ASSERT(client.getType() == PACKET_EOF); - ASSERT(client.eof.header == server.header); - ASSERT(client.eof.warnings == server.warnings); - ASSERT(client.eof.status_flags == server.status_flags); + ASSERT(client.getType() == PACKET_EOF) + ASSERT(client.eof.header == server.header) + ASSERT(client.eof.warnings == server.warnings) + ASSERT(client.eof.status_flags == server.status_flags) } /// ColumnDefinition Packet @@ -149,16 +149,16 @@ int main(int, char **) client.readPayloadImpl(in0); // Check - ASSERT(client.column_type == server.column_type); - ASSERT(client.column_length == server.column_length); - ASSERT(client.next_length == server.next_length); - ASSERT(client.character_set == server.character_set); - ASSERT(client.decimals == server.decimals); - ASSERT(client.name == server.name); - ASSERT(client.org_name == server.org_name); - ASSERT(client.table == server.table); - ASSERT(client.org_table == server.org_table); - ASSERT(client.schema == server.schema); + ASSERT(client.column_type == server.column_type) + ASSERT(client.column_length == server.column_length) + ASSERT(client.next_length == server.next_length) + ASSERT(client.character_set == server.character_set) + ASSERT(client.decimals == server.decimals) + ASSERT(client.name == server.name) + ASSERT(client.org_name == server.org_name) + ASSERT(client.table == server.table) + ASSERT(client.org_table == server.org_table) + ASSERT(client.schema == server.schema) } { From c899c34f03570b5b6c41594f017cfbc34d3e55aa Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 28 Apr 2020 21:07:30 +0800 Subject: [PATCH 038/374] add mysql replication class --- src/Core/MySQLClient.cpp | 57 ++++++----- src/Core/MySQLClient.h | 15 ++- src/Core/MySQLProtocol.h | 140 -------------------------- src/Core/MySQLReplication.cpp | 98 +++++++++++++++++++ src/Core/MySQLReplication.h | 157 ++++++++++++++++++++++++++++++ src/Core/tests/mysql_protocol.cpp | 25 +++-- 6 files changed, 319 insertions(+), 173 deletions(-) create mode 100644 src/Core/MySQLReplication.cpp create mode 100644 src/Core/MySQLReplication.h diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 457da104491..2f8f380f0b6 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -1,4 +1,5 @@ -#include +#include "MySQLClient.h" +#include namespace DB { @@ -10,8 +11,8 @@ namespace ErrorCodes extern const int UNKNOWN_PACKET_FROM_SERVER; } -MySQLClient::MySQLClient(const String & host_, UInt16 port_, const String & user_, const String & password_, const String & database_) - : host(host_), port(port_), user(user_), password(std::move(password_)), database(std::move(database_)) +MySQLClient::MySQLClient(const String & host_, UInt16 port_, const String & user_, const String & password_) + : host(host_), port(port_), user(user_), password(std::move(password_)) { client_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_PLUGIN_AUTH | CLIENT_SECURE_CONNECTION; } @@ -23,14 +24,21 @@ bool MySQLClient::connect() disconnect(); } + const Poco::Timespan connection_timeout(10 * 1e9); + const Poco::Timespan receive_timeout(5 * 1e9); + const Poco::Timespan send_timeout(5 * 1e9); + socket = std::make_unique(); address = DNSResolver::instance().resolveAddress(host, port); - socket->connect(*address); + socket->connect(*address, connection_timeout); + socket->setReceiveTimeout(receive_timeout); + socket->setSendTimeout(send_timeout); + socket->setNoDelay(true); + connected = true; in = std::make_shared(*socket); out = std::make_shared(*socket); packet_sender = std::make_shared(*in, *out, seq); - connected = true; return handshake(); } @@ -51,7 +59,7 @@ bool MySQLClient::handshake() packet_sender->receivePacket(handshake); if (handshake.auth_plugin_name != mysql_native_password) { - throw Exception( + throw MySQLClientError( "Only support " + mysql_native_password + " auth plugin name, but got " + handshake.auth_plugin_name, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); } @@ -97,12 +105,7 @@ bool MySQLClient::writeCommand(char command, String query) return ret; } -bool MySQLClient::ping() -{ - return writeCommand(Command::COM_PING, ""); -} - -bool MySQLClient::registerSlave(UInt32 slave_id) +bool MySQLClient::registerSlaveOnMaster(UInt32 slave_id) { RegisterSlave register_slave(slave_id); packet_sender->sendPacket(register_slave, true); @@ -113,23 +116,33 @@ bool MySQLClient::registerSlave(UInt32 slave_id) if (packet_response.getType() == PACKET_ERR) { last_error = packet_response.err.error_message; + return false; } - return (packet_response.getType() != PACKET_ERR); + return true; } -bool MySQLClient::binlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos) +bool MySQLClient::ping() { + return writeCommand(Command::COM_PING, ""); +} + +bool MySQLClient::requestBinlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos) +{ + if (!registerSlaveOnMaster(slave_id)) + { + return false; + } + BinlogDump binlog_dump(binlog_pos, binlog_file_name, slave_id); packet_sender->sendPacket(binlog_dump, true); + return true; +} - PacketResponse packet_response(client_capability_flags); - packet_sender->receivePacket(packet_response); - packet_sender->resetSequenceId(); - if (packet_response.getType() == PACKET_ERR) - { - last_error = packet_response.err.error_message; - } - return (packet_response.getType() != PACKET_ERR); +BinlogEventPtr MySQLClient::readOneBinlogEvent() +{ + MySQLFlavor mysql; + packet_sender->receivePacket(mysql); + return mysql.binlogEvent(); } String MySQLClient::error() diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 12ebbca45f4..4a7b62e98ee 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -1,5 +1,6 @@ #pragma once #include +#include #include #include #include @@ -15,17 +16,24 @@ namespace DB { using namespace MySQLProtocol; +using namespace MySQLReplication; + +class MySQLClientError : public DB::Exception +{ +public: + using Exception::Exception; +}; class MySQLClient { public: - MySQLClient(const String & host_, UInt16 port_, const String & user_, const String & password_, const String & database_); + MySQLClient(const String & host_, UInt16 port_, const String & user_, const String & password_); bool connect(); void disconnect(); bool ping(); - bool registerSlave(UInt32 slave_id); - bool binlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos); String error(); + bool requestBinlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos); + BinlogEventPtr readOneBinlogEvent(); private: String host; @@ -50,6 +58,7 @@ private: std::shared_ptr packet_sender; bool handshake(); + bool registerSlaveOnMaster(UInt32 slave_id); bool writeCommand(char command, String query); }; diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index db24342846e..129acc9722c 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -1504,144 +1504,4 @@ namespace Replication } } -namespace BinlogEvent -{ - using namespace MySQLProtocol; - - class IBinlogEvent; - using BinlogEventPtr = std::shared_ptr; - - /// http://dev.mysql.com/doc/internals/en/binlog-event-type.html - enum EventType - { - UNKNOW_EVENT = 0, - START_EVENT_V3 = 1, - QUERY_EVENT = 2, - STOP_EVENT = 3, - ROTATE_EVENT = 4, - INT_VAR_EVENT = 5, - LOAD_EVENT = 6, - SLAVE_EVENT = 7, - CREATE_FILE_EVENT = 8, - APPEND_BLOCK_EVENT = 9, - EXEC_LOAD_EVENT = 10, - DELETE_FILE_EVENT = 11, - NEW_LOAD_EVENT = 12, - RAND_EVENT = 13, - USER_VAR_EVENT = 14, - FORMAT_DESCRIPTION_EVENT = 15, - XID_EVENT = 16, - BEGIN_LOAD_QUERY_EVENT = 17, - EXECUTE_LOAD_QUERY_EVENT = 18, - TABLE_MAP_EVENT = 19, - WRITE_ROWS_EVENT_V0 = 20, - UPDATE_ROWS_EVENT_V0 = 21, - DELETE_ROWS_EVENT_V0 = 22, - WRITE_ROWS_EVENT_V1 = 23, - UPDATE_ROWS_EVENT_V1 = 24, - DELETE_ROWS_EVENT_V1 = 25, - INCIDENT_EVENT = 26, - HEARTBEAT_EVENT = 27, - IGNORABLE_EVENT = 28, - ROWS_QUERY_EVENT = 29, - WRITE_ROWS_EVENT_V2 = 30, - UPDATE_ROWS_EVENT_V2 = 31, - DELETE_ROWS_EVENT_V2 = 32, - GTID_EVENT = 33, - ANONYMOUS_GTID_EVENT = 34, - PREVIOUS_GTIDS_EVENT = 35, - TRANSACTION_CONTEXT_EVENT = 36, - VIEW_CHANGE_EVENT = 37, - XA_PREPARE_LOG_EVENT = 38, - - /// MariaDB specific values. They start at 160. - MARIA_ANNOTATE_ROWS_EVENT = 160, - MARIA_BINLOG_CHECKPOINT_EVENT = 161, - MARIA_GTID_EVENT = 162, - MARIA_GTID_LIST_EVENT = 163, - MARIA_START_ENCRYPTION_EVENT = 164, - }; - - class EventHeader - { - public: - UInt32 timestamp; - EventType type; - UInt32 server_id; - UInt32 event_size; - UInt32 log_pos; - UInt16 flags; - - EventHeader() = default; - - void parse(ReadBuffer & payload) - { - payload.readStrict(reinterpret_cast(×tamp), 4); - payload.readStrict(reinterpret_cast(&type), 1); - payload.readStrict(reinterpret_cast(&server_id), 4); - payload.readStrict(reinterpret_cast(&event_size), 4); - payload.readStrict(reinterpret_cast(&log_pos), 4); - payload.readStrict(reinterpret_cast(&flags), 2); - } - }; - - class IBinlogEvent - { - public: - IBinlogEvent() = default; - - virtual bool isValid(); - - virtual void parse(ReadBuffer & payload); - - virtual ~IBinlogEvent(); - }; - - class SampleEvent : public IBinlogEvent - { - public: - SampleEvent() = default; - }; - - class IFlavor - { - public: - virtual String getName() = 0; - - virtual BinlogEventPtr readBinlogEvent(ReadBuffer & payload); - - virtual ~IFlavor() = default; - }; - - class MySQLFlavor : public IFlavor - { - public: - MySQLFlavor() = default; - - BinlogEventPtr readBinlogEvent(ReadBuffer & payload) override - { - UInt16 header = 0; - payload.readStrict(reinterpret_cast(&header), 1); - switch (header) - { - case PACKET_EOF: - throw Exception("Master maybe lost", ErrorCodes::UNKNOWN_EXCEPTION); - case PACKET_ERR: - ERR_Packet err; - err.readPayloadImpl(payload); - throw Exception(err.error_message, ErrorCodes::UNKNOWN_EXCEPTION); - } - - EventHeader event_header; - event_header.parse(payload); - - switch (event_header.type) - { - default: - break; - } - return nullptr; - } - }; -} } diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp new file mode 100644 index 00000000000..3c9a21f90db --- /dev/null +++ b/src/Core/MySQLReplication.cpp @@ -0,0 +1,98 @@ +#include "MySQLReplication.h" + +namespace DB::MySQLReplication +{ +namespace ErrorCodes +{ + extern const int UNKNOWN_EXCEPTION; + extern const int UNEXPECTED_PACKET_FROM_SERVER; +} + +/// https://dev.mysql.com/doc/internals/en/binlog-event-header.html +void EventHeader::parse(ReadBuffer & payload) +{ + payload.readStrict(reinterpret_cast(×tamp), 4); + payload.readStrict(reinterpret_cast(&type), 1); + payload.readStrict(reinterpret_cast(&server_id), 4); + payload.readStrict(reinterpret_cast(&event_size), 4); + payload.readStrict(reinterpret_cast(&log_pos), 4); + payload.readStrict(reinterpret_cast(&flags), 2); +} + + +void EventHeader::dump() +{ + std::cerr << "=== Event Header ===" << std::endl; + std::cerr << "Timestamp: " << this->timestamp << std::endl; + std::cerr << "Event Type: " << this->type << std::endl; + std::cerr << "Server ID: " << this->server_id << std::endl; + std::cerr << "Event Size: " << this->event_size << std::endl; + std::cerr << "Log Pos: " << this->log_pos << std::endl; + std::cerr << "Flags: " << this->flags << std::endl; +} + +/// https://dev.mysql.com/doc/internals/en/format-description-event.html +void FormatDescriptionEvent::parse(ReadBuffer & payload) +{ + payload.readStrict(reinterpret_cast(&binlog_version), 2); + payload.readStrict(reinterpret_cast(server_version.data()), 50); + payload.readStrict(reinterpret_cast(&create_timestamp), 4); + payload.readStrict(reinterpret_cast(&event_header_length), 1); + readNullTerminated(event_type_header_length, payload); +} + +void FormatDescriptionEvent::dump() +{ + std::cerr << "=== FormatDescriptionEvent ===" << std::endl; +} + +/// https://dev.mysql.com/doc/internals/en/rotate-event.html +void RotateEvent::parse(ReadBuffer & payload) +{ + payload.readStrict(reinterpret_cast(&position), 8); + readString(next_binlog, payload); +} + +void RotateEvent::dump() +{ + std::cerr << "=== RotateEvent ===" << std::endl; + std::cerr << "Position: " << this->position << std::endl; + std::cerr << "Next Binlog: " << this->next_binlog << std::endl; +} + +void MySQLFlavor::readPayloadImpl(ReadBuffer & payload) +{ + UInt16 header = static_cast(*payload.position()); + switch (header) + { + case PACKET_EOF: + throw ReplicationError("Master maybe lost", ErrorCodes::UNKNOWN_EXCEPTION); + case PACKET_ERR: + ERR_Packet err; + err.readPayloadImpl(payload); + throw ReplicationError(err.error_message, ErrorCodes::UNKNOWN_EXCEPTION); + } + // skip the header + payload.ignore(1); + + EventHeader event_header; + event_header.parse(payload); + switch (event_header.type) + { + case FORMAT_DESCRIPTION_EVENT: { + event = std::make_shared(FormatDescriptionEvent()); + event->parse(payload); + break; + } + case ROTATE_EVENT: { + event = std::make_shared(RotateEvent()); + event->parse(payload); + break; + } + default: + throw ReplicationError("Unsupported event: " + event_header.type, ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); + break; + } +} + +} diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h new file mode 100644 index 00000000000..d564e201abb --- /dev/null +++ b/src/Core/MySQLReplication.h @@ -0,0 +1,157 @@ +#pragma once +#include +#include +#include +#include + +/// Implementation of MySQL replication protocol. +/// Works only on little-endian architecture. + +namespace DB +{ + +namespace MySQLReplication +{ + using namespace MySQLProtocol; + + class IBinlogEvent; + using BinlogEventPtr = std::shared_ptr; + + /// http://dev.mysql.com/doc/internals/en/binlog-event-type.html + enum EventType + { + UNKNOW_EVENT = 0, + START_EVENT_V3 = 1, + QUERY_EVENT = 2, + STOP_EVENT = 3, + ROTATE_EVENT = 4, + INT_VAR_EVENT = 5, + LOAD_EVENT = 6, + SLAVE_EVENT = 7, + CREATE_FILE_EVENT = 8, + APPEND_BLOCK_EVENT = 9, + EXEC_LOAD_EVENT = 10, + DELETE_FILE_EVENT = 11, + NEW_LOAD_EVENT = 12, + RAND_EVENT = 13, + USER_VAR_EVENT = 14, + FORMAT_DESCRIPTION_EVENT = 15, + XID_EVENT = 16, + BEGIN_LOAD_QUERY_EVENT = 17, + EXECUTE_LOAD_QUERY_EVENT = 18, + TABLE_MAP_EVENT = 19, + WRITE_ROWS_EVENT_V0 = 20, + UPDATE_ROWS_EVENT_V0 = 21, + DELETE_ROWS_EVENT_V0 = 22, + WRITE_ROWS_EVENT_V1 = 23, + UPDATE_ROWS_EVENT_V1 = 24, + DELETE_ROWS_EVENT_V1 = 25, + INCIDENT_EVENT = 26, + HEARTBEAT_EVENT = 27, + IGNORABLE_EVENT = 28, + ROWS_QUERY_EVENT = 29, + WRITE_ROWS_EVENT_V2 = 30, + UPDATE_ROWS_EVENT_V2 = 31, + DELETE_ROWS_EVENT_V2 = 32, + GTID_EVENT = 33, + ANONYMOUS_GTID_EVENT = 34, + PREVIOUS_GTIDS_EVENT = 35, + TRANSACTION_CONTEXT_EVENT = 36, + VIEW_CHANGE_EVENT = 37, + XA_PREPARE_LOG_EVENT = 38, + + /// MariaDB specific values. They start at 160. + MARIA_ANNOTATE_ROWS_EVENT = 160, + MARIA_BINLOG_CHECKPOINT_EVENT = 161, + MARIA_GTID_EVENT = 162, + MARIA_GTID_LIST_EVENT = 163, + MARIA_START_ENCRYPTION_EVENT = 164, + }; + + class ReplicationError : public DB::Exception + { + public: + using Exception::Exception; + }; + + class IBinlogEvent + { + public: + virtual ~IBinlogEvent() = default; + + virtual EventType type() { return UNKNOW_EVENT; } + virtual void parse(ReadBuffer & payload) = 0; + virtual void dump() = 0; + }; + + class EventHeader : public IBinlogEvent + { + public: + UInt32 timestamp; + EventType type = UNKNOW_EVENT; + UInt32 server_id; + UInt32 event_size; + UInt32 log_pos; + UInt16 flags; + + EventHeader() = default; + ~EventHeader() = default; + + void parse(ReadBuffer & payload) override ; + void dump() override ; + }; + + class FormatDescriptionEvent : public IBinlogEvent + { + public: + UInt16 binlog_version; + String server_version; + UInt32 create_timestamp; + UInt8 event_header_length; + String event_type_header_length; + + FormatDescriptionEvent() = default; + ~FormatDescriptionEvent() = default; + + EventType type() override { return FORMAT_DESCRIPTION_EVENT; } + void parse(ReadBuffer & payload) override; + void dump() override ; + }; + + class RotateEvent : public IBinlogEvent + { + public: + UInt64 position; + String next_binlog; + + RotateEvent() = default; + ~RotateEvent() = default; + + EventType type() override { return ROTATE_EVENT; } + void parse(ReadBuffer & payload) override; + void dump() override ; + }; + + class IFlavor + { + public: + virtual String getName() = 0; + virtual BinlogEventPtr binlogEvent() = 0; + virtual ~IFlavor() = default; + }; + + class MySQLFlavor : public IFlavor, public ReadPacket + { + public: + BinlogEventPtr event; + + MySQLFlavor() = default; + ~MySQLFlavor() = default; + + String getName() override { return "MySQL"; } + void readPayloadImpl(ReadBuffer & payload) override; + BinlogEventPtr binlogEvent() override { return event; } + }; +} + +} diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index 8f6d9dfa7a9..ac6a57e3841 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -162,25 +162,34 @@ int main(int, char **) } { - UInt32 slave_id = 123; - MySQLClient client1("127.0.0.1", 9001, "default", "123", ""); - if (!client1.connect()) + /* + UInt32 slave_id = 9004; + MySQLClient slave("127.0.0.1", 9001, "default", "123"); + if (!slave.connect()) { - std::cerr << "Connect Error: " << client1.error() << std::endl; + std::cerr << "Connect Error: " << slave.error() << std::endl; return 1; } - if (!client1.ping()) + if (!slave.ping()) { - std::cerr << "Connect Error: " << client1.error() << std::endl; + std::cerr << "Connect Error: " << slave.error() << std::endl; return 1; } - if (!client1.binlogDump(slave_id, "", 315)) + if (!slave.requestBinlogDump(slave_id, "mysql-bin.000005", 4)) { - std::cerr << "Connect Error: " << client1.error() << std::endl; + std::cerr << "Connect Error: " << slave.error() << std::endl; assert(0); } + + while (true) + { + auto event = slave.readOneBinlogEvent(); + event->dump(); + ASSERT(event != nullptr) + } + */ } return 0; } From 9c2ecf008c5759537c27ed0b801c6eb64878125c Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Wed, 29 Apr 2020 11:26:33 +0800 Subject: [PATCH 039/374] fix clang build --- src/Core/MySQLReplication.cpp | 7 +++---- src/Core/MySQLReplication.h | 17 ++++++++--------- 2 files changed, 11 insertions(+), 13 deletions(-) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 3c9a21f90db..179ffd10781 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -80,18 +80,17 @@ void MySQLFlavor::readPayloadImpl(ReadBuffer & payload) switch (event_header.type) { case FORMAT_DESCRIPTION_EVENT: { - event = std::make_shared(FormatDescriptionEvent()); + event = std::make_shared(); event->parse(payload); break; } case ROTATE_EVENT: { - event = std::make_shared(RotateEvent()); + event = std::make_shared(); event->parse(payload); break; } default: - throw ReplicationError("Unsupported event: " + event_header.type, ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); - break; + throw ReplicationError("Unsupported event: " + std::to_string(event_header.type), ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); } } diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index d564e201abb..48ffe457df8 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -9,7 +9,6 @@ namespace DB { - namespace MySQLReplication { using namespace MySQLProtocol; @@ -84,7 +83,7 @@ namespace MySQLReplication virtual void dump() = 0; }; - class EventHeader : public IBinlogEvent + class EventHeader { public: UInt32 timestamp; @@ -97,8 +96,8 @@ namespace MySQLReplication EventHeader() = default; ~EventHeader() = default; - void parse(ReadBuffer & payload) override ; - void dump() override ; + void parse(ReadBuffer & payload); + void dump() ; }; class FormatDescriptionEvent : public IBinlogEvent @@ -111,11 +110,11 @@ namespace MySQLReplication String event_type_header_length; FormatDescriptionEvent() = default; - ~FormatDescriptionEvent() = default; + ~FormatDescriptionEvent() override = default; EventType type() override { return FORMAT_DESCRIPTION_EVENT; } void parse(ReadBuffer & payload) override; - void dump() override ; + void dump() override; }; class RotateEvent : public IBinlogEvent @@ -125,11 +124,11 @@ namespace MySQLReplication String next_binlog; RotateEvent() = default; - ~RotateEvent() = default; + ~RotateEvent() override = default; EventType type() override { return ROTATE_EVENT; } void parse(ReadBuffer & payload) override; - void dump() override ; + void dump() override; }; class IFlavor @@ -146,7 +145,7 @@ namespace MySQLReplication BinlogEventPtr event; MySQLFlavor() = default; - ~MySQLFlavor() = default; + ~MySQLFlavor() override = default; String getName() override { return "MySQL"; } void readPayloadImpl(ReadBuffer & payload) override; From eaff3836ef364c0d27c0286c01d8167b5921214e Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Wed, 29 Apr 2020 12:13:37 +0800 Subject: [PATCH 040/374] move type() to the base class --- src/Core/MySQLReplication.cpp | 11 +++++------ src/Core/MySQLReplication.h | 33 +++++++++++++++++-------------- src/Core/tests/mysql_protocol.cpp | 2 -- 3 files changed, 23 insertions(+), 23 deletions(-) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 179ffd10781..13762ed5ecc 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -2,11 +2,6 @@ namespace DB::MySQLReplication { -namespace ErrorCodes -{ - extern const int UNKNOWN_EXCEPTION; - extern const int UNEXPECTED_PACKET_FROM_SERVER; -} /// https://dev.mysql.com/doc/internals/en/binlog-event-header.html void EventHeader::parse(ReadBuffer & payload) @@ -43,6 +38,7 @@ void FormatDescriptionEvent::parse(ReadBuffer & payload) void FormatDescriptionEvent::dump() { + header.dump(); std::cerr << "=== FormatDescriptionEvent ===" << std::endl; } @@ -55,6 +51,7 @@ void RotateEvent::parse(ReadBuffer & payload) void RotateEvent::dump() { + header.dump(); std::cerr << "=== RotateEvent ===" << std::endl; std::cerr << "Position: " << this->position << std::endl; std::cerr << "Next Binlog: " << this->next_binlog << std::endl; @@ -82,15 +79,17 @@ void MySQLFlavor::readPayloadImpl(ReadBuffer & payload) case FORMAT_DESCRIPTION_EVENT: { event = std::make_shared(); event->parse(payload); + event->setHeader(event_header); break; } case ROTATE_EVENT: { event = std::make_shared(); event->parse(payload); + event->setHeader(event_header); break; } default: - throw ReplicationError("Unsupported event: " + std::to_string(event_header.type), ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); + throw ReplicationError("Unsupported event: " + std::to_string(event_header.type), ErrorCodes::UNKNOWN_EXCEPTION); } } diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index 48ffe457df8..d2624516d3d 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -73,16 +73,6 @@ namespace MySQLReplication using Exception::Exception; }; - class IBinlogEvent - { - public: - virtual ~IBinlogEvent() = default; - - virtual EventType type() { return UNKNOW_EVENT; } - virtual void parse(ReadBuffer & payload) = 0; - virtual void dump() = 0; - }; - class EventHeader { public: @@ -97,7 +87,22 @@ namespace MySQLReplication ~EventHeader() = default; void parse(ReadBuffer & payload); - void dump() ; + void dump(); + }; + + class IBinlogEvent + { + public: + virtual ~IBinlogEvent() = default; + + virtual void dump() = 0; + virtual void parse(ReadBuffer & payload) = 0; + + EventType type() { return header.type; } + void setHeader(EventHeader header_) { header = header_; } + + protected: + EventHeader header; }; class FormatDescriptionEvent : public IBinlogEvent @@ -112,9 +117,8 @@ namespace MySQLReplication FormatDescriptionEvent() = default; ~FormatDescriptionEvent() override = default; - EventType type() override { return FORMAT_DESCRIPTION_EVENT; } - void parse(ReadBuffer & payload) override; void dump() override; + void parse(ReadBuffer & payload) override; }; class RotateEvent : public IBinlogEvent @@ -126,9 +130,8 @@ namespace MySQLReplication RotateEvent() = default; ~RotateEvent() override = default; - EventType type() override { return ROTATE_EVENT; } - void parse(ReadBuffer & payload) override; void dump() override; + void parse(ReadBuffer & payload) override; }; class IFlavor diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index ac6a57e3841..99d0849381f 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -162,7 +162,6 @@ int main(int, char **) } { - /* UInt32 slave_id = 9004; MySQLClient slave("127.0.0.1", 9001, "default", "123"); if (!slave.connect()) @@ -189,7 +188,6 @@ int main(int, char **) event->dump(); ASSERT(event != nullptr) } - */ } return 0; } From 3c991db64be35ea0626d1d5f13102f84bf5220fa Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Wed, 29 Apr 2020 13:06:44 +0800 Subject: [PATCH 041/374] fix clang expection error --- src/Core/MySQLReplication.cpp | 171 ++++++++++++++++-------------- src/Core/MySQLReplication.h | 18 +--- src/Core/tests/mysql_protocol.cpp | 28 ----- 3 files changed, 92 insertions(+), 125 deletions(-) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 13762ed5ecc..d2ef95a7dce 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -1,96 +1,105 @@ #include "MySQLReplication.h" -namespace DB::MySQLReplication +namespace DB { - -/// https://dev.mysql.com/doc/internals/en/binlog-event-header.html -void EventHeader::parse(ReadBuffer & payload) +namespace ErrorCodes { - payload.readStrict(reinterpret_cast(×tamp), 4); - payload.readStrict(reinterpret_cast(&type), 1); - payload.readStrict(reinterpret_cast(&server_id), 4); - payload.readStrict(reinterpret_cast(&event_size), 4); - payload.readStrict(reinterpret_cast(&log_pos), 4); - payload.readStrict(reinterpret_cast(&flags), 2); + extern const int UNKNOWN_EXCEPTION; } - -void EventHeader::dump() +namespace MySQLReplication { - std::cerr << "=== Event Header ===" << std::endl; - std::cerr << "Timestamp: " << this->timestamp << std::endl; - std::cerr << "Event Type: " << this->type << std::endl; - std::cerr << "Server ID: " << this->server_id << std::endl; - std::cerr << "Event Size: " << this->event_size << std::endl; - std::cerr << "Log Pos: " << this->log_pos << std::endl; - std::cerr << "Flags: " << this->flags << std::endl; -} + using namespace MySQLProtocol; -/// https://dev.mysql.com/doc/internals/en/format-description-event.html -void FormatDescriptionEvent::parse(ReadBuffer & payload) -{ - payload.readStrict(reinterpret_cast(&binlog_version), 2); - payload.readStrict(reinterpret_cast(server_version.data()), 50); - payload.readStrict(reinterpret_cast(&create_timestamp), 4); - payload.readStrict(reinterpret_cast(&event_header_length), 1); - readNullTerminated(event_type_header_length, payload); -} - -void FormatDescriptionEvent::dump() -{ - header.dump(); - std::cerr << "=== FormatDescriptionEvent ===" << std::endl; -} - -/// https://dev.mysql.com/doc/internals/en/rotate-event.html -void RotateEvent::parse(ReadBuffer & payload) -{ - payload.readStrict(reinterpret_cast(&position), 8); - readString(next_binlog, payload); -} - -void RotateEvent::dump() -{ - header.dump(); - std::cerr << "=== RotateEvent ===" << std::endl; - std::cerr << "Position: " << this->position << std::endl; - std::cerr << "Next Binlog: " << this->next_binlog << std::endl; -} - -void MySQLFlavor::readPayloadImpl(ReadBuffer & payload) -{ - UInt16 header = static_cast(*payload.position()); - switch (header) + /// https://dev.mysql.com/doc/internals/en/binlog-event-header.html + void EventHeader::parse(ReadBuffer & payload) { - case PACKET_EOF: - throw ReplicationError("Master maybe lost", ErrorCodes::UNKNOWN_EXCEPTION); - case PACKET_ERR: - ERR_Packet err; - err.readPayloadImpl(payload); - throw ReplicationError(err.error_message, ErrorCodes::UNKNOWN_EXCEPTION); + payload.readStrict(reinterpret_cast(×tamp), 4); + payload.readStrict(reinterpret_cast(&type), 1); + payload.readStrict(reinterpret_cast(&server_id), 4); + payload.readStrict(reinterpret_cast(&event_size), 4); + payload.readStrict(reinterpret_cast(&log_pos), 4); + payload.readStrict(reinterpret_cast(&flags), 2); } - // skip the header - payload.ignore(1); - EventHeader event_header; - event_header.parse(payload); - switch (event_header.type) + + void EventHeader::dump() const { - case FORMAT_DESCRIPTION_EVENT: { - event = std::make_shared(); - event->parse(payload); - event->setHeader(event_header); - break; - } - case ROTATE_EVENT: { - event = std::make_shared(); - event->parse(payload); - event->setHeader(event_header); - break; - } - default: - throw ReplicationError("Unsupported event: " + std::to_string(event_header.type), ErrorCodes::UNKNOWN_EXCEPTION); + std::cerr << "=== Event Header ===" << std::endl; + std::cerr << "Timestamp: " << this->timestamp << std::endl; + std::cerr << "Event Type: " << this->type << std::endl; + std::cerr << "Server ID: " << this->server_id << std::endl; + std::cerr << "Event Size: " << this->event_size << std::endl; + std::cerr << "Log Pos: " << this->log_pos << std::endl; + std::cerr << "Flags: " << this->flags << std::endl; + } + + /// https://dev.mysql.com/doc/internals/en/format-description-event.html + void FormatDescriptionEvent::parse(ReadBuffer & payload) + { + payload.readStrict(reinterpret_cast(&binlog_version), 2); + payload.readStrict(reinterpret_cast(server_version.data()), 50); + payload.readStrict(reinterpret_cast(&create_timestamp), 4); + payload.readStrict(reinterpret_cast(&event_header_length), 1); + readNullTerminated(event_type_header_length, payload); + } + + void FormatDescriptionEvent::dump() + { + header.dump(); + std::cerr << "=== FormatDescriptionEvent ===" << std::endl; + } + + /// https://dev.mysql.com/doc/internals/en/rotate-event.html + void RotateEvent::parse(ReadBuffer & payload) + { + payload.readStrict(reinterpret_cast(&position), 8); + readString(next_binlog, payload); + } + + void RotateEvent::dump() + { + header.dump(); + std::cerr << "=== RotateEvent ===" << std::endl; + std::cerr << "Position: " << this->position << std::endl; + std::cerr << "Next Binlog: " << this->next_binlog << std::endl; + } + + void MySQLFlavor::readPayloadImpl(ReadBuffer & payload) + { + UInt16 header = static_cast(*payload.position()); + switch (header) + { + case PACKET_EOF: + throw ReplicationError("Master maybe lost", ErrorCodes::UNKNOWN_EXCEPTION); + case PACKET_ERR: + ERR_Packet err; + err.readPayloadImpl(payload); + throw ReplicationError(err.error_message, ErrorCodes::UNKNOWN_EXCEPTION); + } + // skip the header + payload.ignore(1); + + EventHeader event_header; + event_header.parse(payload); + switch (event_header.type) + { + case FORMAT_DESCRIPTION_EVENT: { + event = std::make_shared(); + event->parse(payload); + event->setHeader(event_header); + break; + } + case ROTATE_EVENT: { + event = std::make_shared(); + event->parse(payload); + event->setHeader(event_header); + break; + } + default: + throw ReplicationError("Unsupported event: " + std::to_string(event_header.type), ErrorCodes::UNKNOWN_EXCEPTION); + } } -} } +} diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index d2624516d3d..3ed7b6d4c3a 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -11,8 +11,6 @@ namespace DB { namespace MySQLReplication { - using namespace MySQLProtocol; - class IBinlogEvent; using BinlogEventPtr = std::shared_ptr; @@ -83,11 +81,8 @@ namespace MySQLReplication UInt32 log_pos; UInt16 flags; - EventHeader() = default; - ~EventHeader() = default; - + void dump() const; void parse(ReadBuffer & payload); - void dump(); }; class IBinlogEvent @@ -114,9 +109,6 @@ namespace MySQLReplication UInt8 event_header_length; String event_type_header_length; - FormatDescriptionEvent() = default; - ~FormatDescriptionEvent() override = default; - void dump() override; void parse(ReadBuffer & payload) override; }; @@ -127,9 +119,6 @@ namespace MySQLReplication UInt64 position; String next_binlog; - RotateEvent() = default; - ~RotateEvent() override = default; - void dump() override; void parse(ReadBuffer & payload) override; }; @@ -142,14 +131,11 @@ namespace MySQLReplication virtual ~IFlavor() = default; }; - class MySQLFlavor : public IFlavor, public ReadPacket + class MySQLFlavor : public IFlavor, public MySQLProtocol::ReadPacket { public: BinlogEventPtr event; - MySQLFlavor() = default; - ~MySQLFlavor() override = default; - String getName() override { return "MySQL"; } void readPayloadImpl(ReadBuffer & payload) override; BinlogEventPtr binlogEvent() override { return event; } diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index 99d0849381f..fe59c838235 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -161,33 +161,5 @@ int main(int, char **) ASSERT(client.schema == server.schema) } - { - UInt32 slave_id = 9004; - MySQLClient slave("127.0.0.1", 9001, "default", "123"); - if (!slave.connect()) - { - std::cerr << "Connect Error: " << slave.error() << std::endl; - return 1; - } - - if (!slave.ping()) - { - std::cerr << "Connect Error: " << slave.error() << std::endl; - return 1; - } - - if (!slave.requestBinlogDump(slave_id, "mysql-bin.000005", 4)) - { - std::cerr << "Connect Error: " << slave.error() << std::endl; - assert(0); - } - - while (true) - { - auto event = slave.readOneBinlogEvent(); - event->dump(); - ASSERT(event != nullptr) - } - } return 0; } From 79af54c7915f853ee696c62c29a683f1d9c9610a Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Sat, 2 May 2020 16:23:06 +0800 Subject: [PATCH 042/374] add DryRun event --- src/Core/MySQLClient.cpp | 7 +- src/Core/MySQLClient.h | 2 +- src/Core/MySQLReplication.cpp | 169 +++++++++++++++++++++++++++--- src/Core/MySQLReplication.h | 30 ++++-- src/Core/tests/mysql_protocol.cpp | 29 +++++ 5 files changed, 211 insertions(+), 26 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 2f8f380f0b6..14ca69966f3 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -126,13 +126,16 @@ bool MySQLClient::ping() return writeCommand(Command::COM_PING, ""); } -bool MySQLClient::requestBinlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos) +bool MySQLClient::startBinlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos) { + if (!writeCommand(Command::COM_QUERY, "SET @master_binlog_checksum = 'NONE'")) + { + return false; + } if (!registerSlaveOnMaster(slave_id)) { return false; } - BinlogDump binlog_dump(binlog_pos, binlog_file_name, slave_id); packet_sender->sendPacket(binlog_dump, true); return true; diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 4a7b62e98ee..31e4888c885 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -32,7 +32,7 @@ public: void disconnect(); bool ping(); String error(); - bool requestBinlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos); + bool startBinlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos); BinlogEventPtr readOneBinlogEvent(); private: diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index d2ef95a7dce..1a201787a3b 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -11,6 +11,119 @@ namespace MySQLReplication { using namespace MySQLProtocol; + String ToString(BinlogChecksumAlg type) + { + switch (type) + { + case BINLOG_CHECKSUM_ALG_OFF: + return "BINLOG_CHECKSUM_ALG_OFF"; + case BINLOG_CHECKSUM_ALG_CRC32: + return "BINLOG_CHECKSUM_ALG_CRC32"; + case BINLOG_CHECKSUM_ALG_ENUM_END: + return "BINLOG_CHECKSUM_ALG_ENUM_END"; + case BINLOG_CHECKSUM_ALG_UNDEF: + return "BINLOG_CHECKSUM_ALG_UNDEF"; + default: + return std::string("Unknown checksum alg: ") + std::to_string(static_cast(type)); + } + } + + String ToString(EventType type) + { + switch (type) + { + case START_EVENT_V3: + return "StartEventV3"; + case QUERY_EVENT: + return "QueryEvent"; + case STOP_EVENT: + return "StopEvent"; + case ROTATE_EVENT: + return "RotateEvent"; + case INT_VAR_EVENT: + return "IntVarEvent"; + case LOAD_EVENT: + return "LoadEvent"; + case SLAVE_EVENT: + return "SlaveEvent"; + case CREATE_FILE_EVENT: + return "CreateFileEvent"; + case APPEND_BLOCK_EVENT: + return "AppendBlockEvent"; + case EXEC_LOAD_EVENT: + return "ExecLoadEvent"; + case DELETE_FILE_EVENT: + return "DeleteFileEvent"; + case NEW_LOAD_EVENT: + return "NewLoadEvent"; + case RAND_EVENT: + return "RandEvent"; + case USER_VAR_EVENT: + return "UserVarEvent"; + case FORMAT_DESCRIPTION_EVENT: + return "FormatDescriptionEvent"; + case XID_EVENT: + return "XIDEvent"; + case BEGIN_LOAD_QUERY_EVENT: + return "BeginLoadQueryEvent"; + case EXECUTE_LOAD_QUERY_EVENT: + return "ExecuteLoadQueryEvent"; + case TABLE_MAP_EVENT: + return "TableMapEvent"; + case WRITE_ROWS_EVENT_V0: + return "WriteRowsEventV0"; + case UPDATE_ROWS_EVENT_V0: + return "UpdateRowsEventV0"; + case DELETE_ROWS_EVENT_V0: + return "DeleteRowsEventV0"; + case WRITE_ROWS_EVENT_V1: + return "WriteRowsEventV1"; + case UPDATE_ROWS_EVENT_V1: + return "UpdateRowsEventV1"; + case DELETE_ROWS_EVENT_V1: + return "DeleteRowsEventV1"; + case INCIDENT_EVENT: + return "IncidentEvent"; + case HEARTBEAT_EVENT: + return "HeartbeatEvent"; + case IGNORABLE_EVENT: + return "IgnorableEvent"; + case ROWS_QUERY_EVENT: + return "RowsQueryEvent"; + case WRITE_ROWS_EVENT_V2: + return "WriteRowsEventV2"; + case UPDATE_ROWS_EVENT_V2: + return "UpdateRowsEventV2"; + case DELETE_ROWS_EVENT_V2: + return "DeleteRowsEventV2"; + case GTID_EVENT: + return "GTIDEvent"; + case ANONYMOUS_GTID_EVENT: + return "AnonymousGTIDEvent"; + case PREVIOUS_GTIDS_EVENT: + return "PreviousGTIDsEvent"; + case TRANSACTION_CONTEXT_EVENT: + return "TransactionContextEvent"; + case VIEW_CHANGE_EVENT: + return "ViewChangeEvent"; + case XA_PREPARE_LOG_EVENT: + return "XAPrepareLogEvent"; + case MARIA_ANNOTATE_ROWS_EVENT: + return "MariaAnnotateRowsEvent"; + case MARIA_BINLOG_CHECKPOINT_EVENT: + return "MariaBinlogCheckpointEvent"; + case MARIA_GTID_EVENT: + return "MariaGTIDEvent"; + case MARIA_GTID_LIST_EVENT: + return "MariaGTIDListEvent"; + case MARIA_START_ENCRYPTION_EVENT: + return "MariaStartEncryptionEvent"; + default: + break; + } + return std::string("Unknown event: ") + std::to_string(static_cast(type)); + } + /// https://dev.mysql.com/doc/internals/en/binlog-event-header.html void EventHeader::parse(ReadBuffer & payload) { @@ -22,10 +135,9 @@ namespace MySQLReplication payload.readStrict(reinterpret_cast(&flags), 2); } - void EventHeader::dump() const { - std::cerr << "=== Event Header ===" << std::endl; + std::cerr << "\n=== " << ToString(this->type) << " ===" << std::endl; std::cerr << "Timestamp: " << this->timestamp << std::endl; std::cerr << "Event Type: " << this->type << std::endl; std::cerr << "Server ID: " << this->server_id << std::endl; @@ -38,33 +150,57 @@ namespace MySQLReplication void FormatDescriptionEvent::parse(ReadBuffer & payload) { payload.readStrict(reinterpret_cast(&binlog_version), 2); + assert(binlog_version == EVENT_VERSION); payload.readStrict(reinterpret_cast(server_version.data()), 50); payload.readStrict(reinterpret_cast(&create_timestamp), 4); payload.readStrict(reinterpret_cast(&event_header_length), 1); - readNullTerminated(event_type_header_length, payload); + assert(event_header_length == EVENT_HEADER_LENGTH); + + size_t len = header.event_size - (2 + 50 + 4 + 1 + EVENT_HEADER_LENGTH) - 1; + event_type_header_length.resize(len); + payload.readStrict(reinterpret_cast(event_type_header_length.data()), len); + payload.readStrict(reinterpret_cast(&checksum_alg), 1); } void FormatDescriptionEvent::dump() { header.dump(); - std::cerr << "=== FormatDescriptionEvent ===" << std::endl; + std::cerr << "Binlog Version: " << this->binlog_version << std::endl; + std::cerr << "Server Version: " << this->server_version << std::endl; + std::cerr << "Create Timestamp: " << this->create_timestamp << std::endl; + std::cerr << "Event Header Len: " << this->event_header_length << std::endl; + std::cerr << "Binlog Checksum Alg: " << ToString(this->checksum_alg) << std::endl; } /// https://dev.mysql.com/doc/internals/en/rotate-event.html void RotateEvent::parse(ReadBuffer & payload) { payload.readStrict(reinterpret_cast(&position), 8); - readString(next_binlog, payload); + size_t len = header.event_size - EVENT_HEADER_LENGTH - 8; + next_binlog.resize(len); + payload.readStrict(reinterpret_cast(next_binlog.data()), len); } void RotateEvent::dump() { header.dump(); - std::cerr << "=== RotateEvent ===" << std::endl; std::cerr << "Position: " << this->position << std::endl; std::cerr << "Next Binlog: " << this->next_binlog << std::endl; } + void DryRunEvent::parse(ReadBuffer & payload) + { + while (payload.next()) + { + } + } + + void DryRunEvent::dump() + { + header.dump(); + std::cerr << "[DryRun Event]" << std::endl; + } + void MySQLFlavor::readPayloadImpl(ReadBuffer & payload) { UInt16 header = static_cast(*payload.position()); @@ -77,29 +213,28 @@ namespace MySQLReplication err.readPayloadImpl(payload); throw ReplicationError(err.error_message, ErrorCodes::UNKNOWN_EXCEPTION); } - // skip the header + // skip the header flag. payload.ignore(1); - EventHeader event_header; - event_header.parse(payload); - switch (event_header.type) + EventType event_type = static_cast(*(payload.position() + 4)); + switch (event_type) { case FORMAT_DESCRIPTION_EVENT: { event = std::make_shared(); - event->parse(payload); - event->setHeader(event_header); break; } case ROTATE_EVENT: { event = std::make_shared(); - event->parse(payload); - event->setHeader(event_header); break; } - default: - throw ReplicationError("Unsupported event: " + std::to_string(event_header.type), ErrorCodes::UNKNOWN_EXCEPTION); + default: { + event = std::make_shared(); + break; + } } + event->header.parse(payload); + event->parse(payload); } +} } -} diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index 3ed7b6d4c3a..f41df6a2c26 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -11,9 +11,21 @@ namespace DB { namespace MySQLReplication { + static const int EVENT_VERSION = 4; + static const int EVENT_HEADER_LENGTH = 19; + class IBinlogEvent; using BinlogEventPtr = std::shared_ptr; + enum BinlogChecksumAlg + { + BINLOG_CHECKSUM_ALG_OFF = 0, + BINLOG_CHECKSUM_ALG_CRC32 = 1, + BINLOG_CHECKSUM_ALG_ENUM_END, + BINLOG_CHECKSUM_ALG_UNDEF = 255 + }; + String ToString(BinlogChecksumAlg type); + /// http://dev.mysql.com/doc/internals/en/binlog-event-type.html enum EventType { @@ -64,6 +76,7 @@ namespace MySQLReplication MARIA_GTID_LIST_EVENT = 163, MARIA_START_ENCRYPTION_EVENT = 164, }; + String ToString(EventType type); class ReplicationError : public DB::Exception { @@ -88,16 +101,13 @@ namespace MySQLReplication class IBinlogEvent { public: - virtual ~IBinlogEvent() = default; + EventHeader header; + virtual ~IBinlogEvent() = default; virtual void dump() = 0; virtual void parse(ReadBuffer & payload) = 0; EventType type() { return header.type; } - void setHeader(EventHeader header_) { header = header_; } - - protected: - EventHeader header; }; class FormatDescriptionEvent : public IBinlogEvent @@ -108,6 +118,7 @@ namespace MySQLReplication UInt32 create_timestamp; UInt8 event_header_length; String event_type_header_length; + BinlogChecksumAlg checksum_alg; void dump() override; void parse(ReadBuffer & payload) override; @@ -123,6 +134,12 @@ namespace MySQLReplication void parse(ReadBuffer & payload) override; }; + class DryRunEvent : public IBinlogEvent + { + void dump() override; + void parse(ReadBuffer & payload) override; + }; + class IFlavor { public: @@ -134,7 +151,8 @@ namespace MySQLReplication class MySQLFlavor : public IFlavor, public MySQLProtocol::ReadPacket { public: - BinlogEventPtr event; + BinlogEventPtr event; + BinlogChecksumAlg binlogChecksumAlg = BINLOG_CHECKSUM_ALG_UNDEF; String getName() override { return "MySQL"; } void readPayloadImpl(ReadBuffer & payload) override; diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index fe59c838235..ba025dc990f 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -161,5 +161,34 @@ int main(int, char **) ASSERT(client.schema == server.schema) } + { + UInt32 slave_id = 9004; + MySQLClient slave("127.0.0.1", 9001, "default", "123"); + if (!slave.connect()) + { + std::cerr << "Connect Error: " << slave.error() << std::endl; + return 1; + } + + if (!slave.ping()) + { + std::cerr << "Connect Error: " << slave.error() << std::endl; + return 1; + } + + if (!slave.startBinlogDump(slave_id, "mysql-bin.000005", 4)) + { + std::cerr << "Connect Error: " << slave.error() << std::endl; + assert(0); + } + + while (true) + { + auto event = slave.readOneBinlogEvent(); + event->dump(); + ASSERT(event != nullptr) + } + } + return 0; } From 02011ae42f43ea82a015889ed40ee609695b3e55 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Sat, 9 May 2020 16:14:02 +0800 Subject: [PATCH 043/374] Add binlog event parser --- src/Core/MySQLClient.cpp | 24 +- src/Core/MySQLClient.h | 11 +- src/Core/MySQLProtocol.h | 5 + src/Core/MySQLReplication.cpp | 384 ++++++++++++++++++++++++++++-- src/Core/MySQLReplication.h | 191 ++++++++++++--- src/Core/tests/mysql_protocol.cpp | 6 +- 6 files changed, 563 insertions(+), 58 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 14ca69966f3..b7166701e99 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -68,7 +68,7 @@ bool MySQLClient::handshake() String auth_plugin_data = native41.getAuthPluginData(); HandshakeResponse handshake_response( - client_capability_flags, max_packet_size, charset_utf8, user, database, auth_plugin_data, mysql_native_password); + client_capability_flags, max_packet_size, charset_utf8, user, "", auth_plugin_data, mysql_native_password); packet_sender->sendPacket(handshake_response, true); PacketResponse packet_response(client_capability_flags); @@ -128,14 +128,25 @@ bool MySQLClient::ping() bool MySQLClient::startBinlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos) { - if (!writeCommand(Command::COM_QUERY, "SET @master_binlog_checksum = 'NONE'")) + String checksum = "CRC32"; + if (!writeCommand(Command::COM_QUERY, "SET @master_binlog_checksum = '" + checksum + "'")) { return false; } + + /// 30s. + UInt64 period_ns = (30 * 1e9); + if (!writeCommand(Command::COM_QUERY, "SET @master_heartbeat_period = " + std::to_string(period_ns))) + { + return false; + } + if (!registerSlaveOnMaster(slave_id)) { return false; } + + binlog_pos = binlog_pos < 4 ? 4 : binlog_pos; BinlogDump binlog_dump(binlog_pos, binlog_file_name, slave_id); packet_sender->sendPacket(binlog_dump, true); return true; @@ -143,13 +154,16 @@ bool MySQLClient::startBinlogDump(UInt32 slave_id, String binlog_file_name, UInt BinlogEventPtr MySQLClient::readOneBinlogEvent() { - MySQLFlavor mysql; - packet_sender->receivePacket(mysql); - return mysql.binlogEvent(); + while (true) + { + packet_sender->receivePacket(replication); + return replication.readOneEvent(); + } } String MySQLClient::error() { return last_error; } + } diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 31e4888c885..58caf1d91e2 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -33,24 +33,25 @@ public: bool ping(); String error(); bool startBinlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos); - BinlogEventPtr readOneBinlogEvent(); + BinlogEventPtr readOneBinlogEvent() ; + Position getPosition() const { return replication.getPosition(); }; private: String host; UInt16 port; String user; String password; - String database; bool connected = false; String last_error; UInt32 client_capability_flags = 0; uint8_t seq = 0; - UInt8 charset_utf8 = 33; - UInt32 max_packet_size = MySQLProtocol::MAX_PACKET_LENGTH; - String mysql_native_password = "mysql_native_password"; + const UInt8 charset_utf8 = 33; + const UInt32 max_packet_size = MySQLProtocol::MAX_PACKET_LENGTH; + const String mysql_native_password = "mysql_native_password"; + MySQLFlavor replication; std::shared_ptr in; std::shared_ptr out; std::unique_ptr socket; diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 129acc9722c..ae97eb3dfe1 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -124,8 +124,13 @@ enum ColumnType MYSQL_TYPE_TIME = 0x0b, MYSQL_TYPE_DATETIME = 0x0c, MYSQL_TYPE_YEAR = 0x0d, + MYSQL_TYPE_NEWDATE = 0x0e, MYSQL_TYPE_VARCHAR = 0x0f, MYSQL_TYPE_BIT = 0x10, + MYSQL_TYPE_TIMESTAMP2 = 0x11, + MYSQL_TYPE_DATETIME2 = 0x12, + MYSQL_TYPE_TIME2 = 0x13, + MYSQL_TYPE_JSON = 0xf5, MYSQL_TYPE_NEWDECIMAL = 0xf6, MYSQL_TYPE_ENUM = 0xf7, MYSQL_TYPE_SET = 0xf8, diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 1a201787a3b..1b7eeb7d53b 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -1,5 +1,8 @@ #include "MySQLReplication.h" +#include +#include + namespace DB { namespace ErrorCodes @@ -135,7 +138,7 @@ namespace MySQLReplication payload.readStrict(reinterpret_cast(&flags), 2); } - void EventHeader::dump() const + void EventHeader::print() const { std::cerr << "\n=== " << ToString(this->type) << " ===" << std::endl; std::cerr << "Timestamp: " << this->timestamp << std::endl; @@ -147,10 +150,10 @@ namespace MySQLReplication } /// https://dev.mysql.com/doc/internals/en/format-description-event.html - void FormatDescriptionEvent::parse(ReadBuffer & payload) + void FormatDescriptionEvent::parseImpl(ReadBuffer & payload) { payload.readStrict(reinterpret_cast(&binlog_version), 2); - assert(binlog_version == EVENT_VERSION); + assert(binlog_version == EVENT_VERSION_V4); payload.readStrict(reinterpret_cast(server_version.data()), 50); payload.readStrict(reinterpret_cast(&create_timestamp), 4); payload.readStrict(reinterpret_cast(&event_header_length), 1); @@ -159,45 +162,341 @@ namespace MySQLReplication size_t len = header.event_size - (2 + 50 + 4 + 1 + EVENT_HEADER_LENGTH) - 1; event_type_header_length.resize(len); payload.readStrict(reinterpret_cast(event_type_header_length.data()), len); - payload.readStrict(reinterpret_cast(&checksum_alg), 1); + assert(event_type_header_length[WRITE_ROWS_EVENT_V2] == ROWS_HEADER_LEN_V2); + assert(event_type_header_length[UPDATE_ROWS_EVENT_V2] == ROWS_HEADER_LEN_V2); + assert(event_type_header_length[DELETE_ROWS_EVENT_V2] == ROWS_HEADER_LEN_V2); } - void FormatDescriptionEvent::dump() + void FormatDescriptionEvent::print() const { - header.dump(); + header.print(); std::cerr << "Binlog Version: " << this->binlog_version << std::endl; std::cerr << "Server Version: " << this->server_version << std::endl; std::cerr << "Create Timestamp: " << this->create_timestamp << std::endl; std::cerr << "Event Header Len: " << this->event_header_length << std::endl; - std::cerr << "Binlog Checksum Alg: " << ToString(this->checksum_alg) << std::endl; } /// https://dev.mysql.com/doc/internals/en/rotate-event.html - void RotateEvent::parse(ReadBuffer & payload) + void RotateEvent::parseImpl(ReadBuffer & payload) { payload.readStrict(reinterpret_cast(&position), 8); - size_t len = header.event_size - EVENT_HEADER_LENGTH - 8; + size_t len = header.event_size - EVENT_HEADER_LENGTH - 8 - CHECKSUM_CRC32_SIGNATURE_LENGTH; next_binlog.resize(len); payload.readStrict(reinterpret_cast(next_binlog.data()), len); } - void RotateEvent::dump() + void RotateEvent::print() const { - header.dump(); + header.print(); std::cerr << "Position: " << this->position << std::endl; std::cerr << "Next Binlog: " << this->next_binlog << std::endl; } - void DryRunEvent::parse(ReadBuffer & payload) + /// https://dev.mysql.com/doc/internals/en/query-event.html + void QueryEvent::parseImpl(ReadBuffer & payload) + { + payload.readStrict(reinterpret_cast(&thread_id), 4); + payload.readStrict(reinterpret_cast(&exec_time), 4); + payload.readStrict(reinterpret_cast(&schema_len), 1); + payload.readStrict(reinterpret_cast(&error_code), 2); + payload.readStrict(reinterpret_cast(&status_len), 2); + + status.resize(status_len); + payload.readStrict(reinterpret_cast(status.data()), status_len); + + schema.resize(schema_len); + payload.readStrict(reinterpret_cast(schema.data()), schema_len); + payload.ignore(1); + + size_t len + = header.event_size - EVENT_HEADER_LENGTH - 4 - 4 - 1 - 2 - 2 - status_len - schema_len - 1 - CHECKSUM_CRC32_SIGNATURE_LENGTH; + query.resize(len); + payload.readStrict(reinterpret_cast(query.data()), len); + } + + void QueryEvent::print() const + { + header.print(); + std::cerr << "Thread ID: " << this->thread_id << std::endl; + std::cerr << "Execution Time: " << this->exec_time << std::endl; + std::cerr << "Schema Len: " << this->schema_len << std::endl; + std::cerr << "Error Code: " << this->error_code << std::endl; + std::cerr << "Status Len: " << this->status_len << std::endl; + std::cerr << "Schema: " << this->schema << std::endl; + std::cerr << "Query: " << this->query << std::endl; + } + + void XIDEvent::parseImpl(ReadBuffer & payload) { payload.readStrict(reinterpret_cast(&xid), 8); } + + + void XIDEvent::print() const + { + header.print(); + std::cerr << "XID: " << this->xid << std::endl; + } + + void TableMapEvent::parseImpl(ReadBuffer & payload) + { + payload.readStrict(reinterpret_cast(&table_id), 6); + payload.readStrict(reinterpret_cast(&flags), 2); + + payload.readStrict(reinterpret_cast(&schema_len), 1); + schema.resize(schema_len); + payload.readStrict(reinterpret_cast(schema.data()), schema_len); + payload.ignore(1); + + payload.readStrict(reinterpret_cast(&table_len), 1); + table.resize(table_len); + payload.readStrict(reinterpret_cast(table.data()), table_len); + payload.ignore(1); + + column_count = readLengthEncodedNumber(payload); + for (auto i = 0U; i < column_count; i++) + { + UInt8 v = 0x00; + payload.readStrict(reinterpret_cast(&v), 1); + column_type.emplace_back(v); + } + + String meta; + readLengthEncodedString(meta, payload); + parseMeta(meta); + + size_t len = (column_count + 8) / 7; + payload.readStrict(reinterpret_cast(null_bitmap.data()), len); + } + + void TableMapEvent::parseMeta(String meta) + { + auto pos = 0; + column_meta.reserve(column_count); + for (auto i = 0U; i < column_count; i++) + { + UInt16 typ = column_type[i]; + switch (typ) + { + case MYSQL_TYPE_DECIMAL: + case MYSQL_TYPE_TINY: + case MYSQL_TYPE_SHORT: + case MYSQL_TYPE_LONG: + case MYSQL_TYPE_NULL: + case MYSQL_TYPE_TIMESTAMP: + case MYSQL_TYPE_LONGLONG: + case MYSQL_TYPE_INT24: + case MYSQL_TYPE_DATE: + case MYSQL_TYPE_TIME: + case MYSQL_TYPE_DATETIME: + case MYSQL_TYPE_YEAR: + case MYSQL_TYPE_NEWDATE: { + /// No data here. + column_meta.emplace_back(0); + break; + } + + case MYSQL_TYPE_FLOAT: + case MYSQL_TYPE_DOUBLE: + case MYSQL_TYPE_TIMESTAMP2: + case MYSQL_TYPE_DATETIME2: + case MYSQL_TYPE_TIME2: + case MYSQL_TYPE_JSON: + case MYSQL_TYPE_TINY_BLOB: + case MYSQL_TYPE_MEDIUM_BLOB: + case MYSQL_TYPE_LONG_BLOB: + case MYSQL_TYPE_BLOB: + case MYSQL_TYPE_GEOMETRY: { + column_meta.emplace_back(UInt16(meta[pos])); + pos += 1; + break; + } + case MYSQL_TYPE_NEWDECIMAL: + case MYSQL_TYPE_ENUM: + case MYSQL_TYPE_SET: + case MYSQL_TYPE_STRING: { + column_meta.emplace_back((UInt16(meta[pos]) << 8) + UInt16(meta[pos + 1])); + pos += 2; + break; + } + + case MYSQL_TYPE_VARCHAR: + case MYSQL_TYPE_BIT: + case MYSQL_TYPE_VAR_STRING: { + column_meta.emplace_back(UInt16(meta[pos]) + (UInt16(meta[pos + 1] << 8))); + pos += 2; + break; + } + default: + throw ReplicationError("ParseMetaData: Unhandled data type:" + std::to_string(typ), ErrorCodes::UNKNOWN_EXCEPTION); + } + } + } + + void TableMapEvent::print() const + { + header.print(); + std::cerr << "Table ID: " << this->table_id << std::endl; + std::cerr << "Flags: " << this->flags << std::endl; + std::cerr << "Schema Len: " << this->schema_len << std::endl; + std::cerr << "Schema: " << this->schema << std::endl; + std::cerr << "Table Len: " << this->table_len << std::endl; + std::cerr << "Table: " << this->table << std::endl; + std::cerr << "Column Count: " << this->column_count << std::endl; + for (auto i = 0U; i < column_count; i++) + { + std::cerr << "Column Type [" << i << "]: " << column_type[i] << ", Meta: " << column_meta[i] << std::endl; + } + std::cerr << "Null Bitmap: " << this->null_bitmap << std::endl; + } + + void RowsEvent::parseImpl(ReadBuffer & payload) + { + payload.readStrict(reinterpret_cast(&table_id), 6); + payload.readStrict(reinterpret_cast(&flags), 2); + + /// This extra_data_len contains the 2 bytes length. + payload.readStrict(reinterpret_cast(&extra_data_len), 2); + payload.ignore(extra_data_len - 2); + + number_columns = readLengthEncodedNumber(payload); + size_t columns_bitmap_size = (number_columns + 8) / 7; + payload.readStrict(reinterpret_cast(columns_before_bitmap.data()), columns_bitmap_size); + if (header.type == UPDATE_ROWS_EVENT_V2) + { + payload.readStrict(reinterpret_cast(columns_after_bitmap.data()), columns_bitmap_size); + } + + while (payload.available() > CHECKSUM_CRC32_SIGNATURE_LENGTH) + { + parseRow(payload, columns_before_bitmap); + if (header.type == UPDATE_ROWS_EVENT_V2) + { + parseRow(payload, columns_after_bitmap); + } + } + } + + void RowsEvent::parseRow(ReadBuffer & payload, String bitmap) + { + UInt32 field_type = 0; + UInt32 field_len = 0; + + bitmap = ""; + size_t columns_null_bitmap_size = (number_columns + 8) / 7; + String columns_null_bitmap; + columns_null_bitmap.resize(columns_null_bitmap_size); + payload.readStrict(reinterpret_cast(columns_null_bitmap.data()), columns_null_bitmap_size); + + Tuple row; + for (auto i = 0U; i < number_columns; i++) + { + field_type = table_map->column_type[i]; + UInt16 meta = table_map->column_meta[i]; + + if (field_type == MYSQL_TYPE_STRING) + { + if (meta >= 256) + { + UInt32 byte0 = meta >> 8; + UInt32 byte1 = meta & 0xff; + if ((byte0 & 0x30) != 0x30) + { + field_len = byte1 | (((byte0 & 0x30) ^ 0x30) << 4); + field_type = byte0 | 0x30; + } + else + { + switch (byte0) + { + case MYSQL_TYPE_SET: + case MYSQL_TYPE_ENUM: + case MYSQL_TYPE_STRING: + field_type = byte0; + field_len = byte1; + break; + default: + throw ReplicationError("ParseRow: Illegal event", ErrorCodes::UNKNOWN_EXCEPTION); + } + } + } + else + { + field_len = meta; + } + } + + switch (field_type) + { + case MYSQL_TYPE_LONG: { + UInt32 val = 0; + payload.readStrict(reinterpret_cast(&val), 4); + row.push_back(Field{UInt32{val}}); + break; + } + case MYSQL_TYPE_VARCHAR: + case MYSQL_TYPE_VAR_STRING: { + uint32_t size = meta; + if (size < 256) + { + uint8_t tmp1 = 0; + payload.readStrict(reinterpret_cast(&tmp1), 1); + size = tmp1; + } + else + { + uint16_t tmp2 = 0; + payload.readStrict(reinterpret_cast(&tmp2), 2); + size = tmp2; + } + + String val; + val.resize(size); + payload.readStrict(reinterpret_cast(val.data()), size); + row.push_back(Field{String{val}}); + break; + } + case MYSQL_TYPE_STRING: { + UInt32 size = field_len; + if (size < 256) + { + uint8_t tmp1 = 0; + payload.readStrict(reinterpret_cast(&tmp1), 1); + size = tmp1; + } + else + { + uint16_t tmp2 = 0; + payload.readStrict(reinterpret_cast(&tmp2), 2); + size = tmp2; + } + break; + } + } + } + rows.push_back(row); + } + + void RowsEvent::print() const + { + FieldVisitorToString to_string; + + header.print(); + std::cerr << "Schema: " << this->schema << std::endl; + std::cerr << "Table: " << this->table << std::endl; + for (auto i = 0U; i < rows.size(); i++) + { + std::cerr << "Row[" << i << "]: " << applyVisitor(to_string, rows[i]) << std::endl; + } + } + + void DryRunEvent::parseImpl(ReadBuffer & payload) { while (payload.next()) { } } - void DryRunEvent::dump() + void DryRunEvent::print() const { - header.dump(); + header.print(); std::cerr << "[DryRun Event]" << std::endl; } @@ -221,19 +520,72 @@ namespace MySQLReplication { case FORMAT_DESCRIPTION_EVENT: { event = std::make_shared(); + event->parseHeader(payload); + event->parseEvent(payload); + position.updateLogPos(event->header.log_pos); break; } case ROTATE_EVENT: { event = std::make_shared(); + event->parseHeader(payload); + event->parseEvent(payload); + + auto rotate = std::dynamic_pointer_cast(event); + position.updateLogPos(event->header.log_pos); + position.updateLogName(rotate->next_binlog); + break; + } + case QUERY_EVENT: { + event = std::make_shared(); + event->parseHeader(payload); + event->parseEvent(payload); + if (event->header.event_size > QUERY_EVENT_BEGIN_LENGTH) + position.updateLogPos(event->header.log_pos); + break; + } + case XID_EVENT: { + event = std::make_shared(); + event->parseHeader(payload); + event->parseEvent(payload); + position.updateLogPos(event->header.log_pos); + break; + } + case TABLE_MAP_EVENT: { + event = std::make_shared(); + event->parseHeader(payload); + event->parseEvent(payload); + + table_map = std::dynamic_pointer_cast(event); + position.updateLogPos(event->header.log_pos); + break; + } + case WRITE_ROWS_EVENT_V2: { + event = std::make_shared(table_map); + event->parseHeader(payload); + event->parseEvent(payload); + break; + } + case DELETE_ROWS_EVENT_V2: { + event = std::make_shared(table_map); + event->parseHeader(payload); + event->parseEvent(payload); + break; + } + case UPDATE_ROWS_EVENT_V2: { + event = std::make_shared(table_map); + event->parseHeader(payload); + event->parseEvent(payload); break; } default: { event = std::make_shared(); + event->parseHeader(payload); + event->parseEvent(payload); + position.updateLogPos(event->header.log_pos); break; } } - event->header.parse(payload); - event->parse(payload); + payload.tryIgnore(CHECKSUM_CRC32_SIGNATURE_LENGTH); } } diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index f41df6a2c26..f41a946f631 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -1,9 +1,12 @@ #pragma once +#include #include #include #include #include +#include + /// Implementation of MySQL replication protocol. /// Works only on little-endian architecture. @@ -11,11 +14,14 @@ namespace DB { namespace MySQLReplication { - static const int EVENT_VERSION = 4; + static const int EVENT_VERSION_V4 = 4; static const int EVENT_HEADER_LENGTH = 19; + static const int CHECKSUM_CRC32_SIGNATURE_LENGTH = 4; + static const int QUERY_EVENT_BEGIN_LENGTH = 74; + static const int ROWS_HEADER_LEN_V2 = 10; - class IBinlogEvent; - using BinlogEventPtr = std::shared_ptr; + class EventBase; + using BinlogEventPtr = std::shared_ptr; enum BinlogChecksumAlg { @@ -94,23 +100,28 @@ namespace MySQLReplication UInt32 log_pos; UInt16 flags; - void dump() const; + void print() const; void parse(ReadBuffer & payload); }; - class IBinlogEvent + class EventBase { public: EventHeader header; - virtual ~IBinlogEvent() = default; - virtual void dump() = 0; - virtual void parse(ReadBuffer & payload) = 0; + virtual ~EventBase() = default; + virtual void print() const = 0; + virtual void parseHeader(ReadBuffer & payload) { header.parse(payload); } - EventType type() { return header.type; } + virtual void parseEvent(ReadBuffer & payload) { parseImpl(payload); } + + EventType type() const { return header.type; } + + protected: + virtual void parseImpl(ReadBuffer & payload) = 0; }; - class FormatDescriptionEvent : public IBinlogEvent + class FormatDescriptionEvent : public EventBase { public: UInt16 binlog_version; @@ -118,45 +129,165 @@ namespace MySQLReplication UInt32 create_timestamp; UInt8 event_header_length; String event_type_header_length; - BinlogChecksumAlg checksum_alg; - void dump() override; - void parse(ReadBuffer & payload) override; + void print() const override; + + protected: + void parseImpl(ReadBuffer & payload) override; + + private: + std::vector post_header_lens; }; - class RotateEvent : public IBinlogEvent + class RotateEvent : public EventBase { public: UInt64 position; String next_binlog; - void dump() override; - void parse(ReadBuffer & payload) override; + void print() const override; + + protected: + void parseImpl(ReadBuffer & payload) override; }; - class DryRunEvent : public IBinlogEvent - { - void dump() override; - void parse(ReadBuffer & payload) override; - }; - - class IFlavor + class QueryEvent : public EventBase { public: - virtual String getName() = 0; - virtual BinlogEventPtr binlogEvent() = 0; + UInt32 thread_id; + UInt32 exec_time; + UInt8 schema_len; + UInt16 error_code; + UInt16 status_len; + String status; + String schema; + String query; + + void print() const override; + + protected: + void parseImpl(ReadBuffer & payload) override; + }; + + class XIDEvent : public EventBase + { + public: + UInt64 xid; + + void print() const override; + + protected: + void parseImpl(ReadBuffer & payload) override; + }; + + class TableMapEvent : public EventBase + { + public: + UInt64 table_id; + UInt16 flags; + UInt8 schema_len; + String schema; + UInt8 table_len; + String table; + UInt32 column_count; + std::vector column_type; + std::vector column_meta; + String null_bitmap; + + void print() const override; + + protected: + void parseImpl(ReadBuffer & payload) override; + void parseMeta(String meta); + }; + + class RowsEvent : public EventBase + { + public: + UInt64 table_id; + UInt16 flags; + UInt16 extra_data_len; + UInt32 number_columns; + String schema; + String table; + String columns_before_bitmap; + String columns_after_bitmap; + std::vector rows; + + RowsEvent(std::shared_ptr table_map_) : table_map(table_map_) + { + schema = table_map->schema; + table = table_map->table; + } + void print() const override; + + protected: + void parseImpl(ReadBuffer & payload) override; + void parseRow(ReadBuffer & payload, String bitmap); + + private: + std::shared_ptr table_map; + }; + + class WriteRowsEvent : public RowsEvent + { + public: + WriteRowsEvent(std::shared_ptr table_map_) : RowsEvent(table_map_) { } + }; + + class DeleteRowsEvent : public RowsEvent + { + public: + DeleteRowsEvent(std::shared_ptr table_map_) : RowsEvent(table_map_) { } + }; + + class UpdateRowsEvent : public RowsEvent + { + public: + UpdateRowsEvent(std::shared_ptr table_map_) : RowsEvent(table_map_) { } + }; + + class DryRunEvent : public EventBase + { + void print() const override; + + protected: + void parseImpl(ReadBuffer & payload) override; + }; + + class Position + { + public: + UInt64 binlog_pos; + String binlog_name; + + Position() : binlog_pos(0), binlog_name("") { } + void updateLogPos(UInt64 pos) { binlog_pos = pos; } + void updateLogName(String binlog) { binlog_name = std::move(binlog); } + }; + + class IFlavor : public MySQLProtocol::ReadPacket + { + public: + virtual String getName() const = 0; + virtual Position getPosition() const = 0; + virtual BinlogEventPtr readOneEvent() = 0; virtual ~IFlavor() = default; }; - class MySQLFlavor : public IFlavor, public MySQLProtocol::ReadPacket + class MySQLFlavor : public IFlavor { public: - BinlogEventPtr event; - BinlogChecksumAlg binlogChecksumAlg = BINLOG_CHECKSUM_ALG_UNDEF; + BinlogEventPtr event; - String getName() override { return "MySQL"; } + String getName() const override { return "MySQL"; } + Position getPosition() const override { return position; } void readPayloadImpl(ReadBuffer & payload) override; - BinlogEventPtr binlogEvent() override { return event; } + BinlogEventPtr readOneEvent() override { return event; } + + private: + Position position; + std::shared_ptr table_map; }; } diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index ba025dc990f..1c6b61b742e 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -176,7 +176,7 @@ int main(int, char **) return 1; } - if (!slave.startBinlogDump(slave_id, "mysql-bin.000005", 4)) + if (!slave.startBinlogDump(slave_id, "", 4)) { std::cerr << "Connect Error: " << slave.error() << std::endl; assert(0); @@ -185,8 +185,10 @@ int main(int, char **) while (true) { auto event = slave.readOneBinlogEvent(); - event->dump(); ASSERT(event != nullptr) + event->print(); + std::cerr << "Binlog Name: " << slave.getPosition().binlog_name << std::endl; + std::cerr << "Binlog Pos: " << slave.getPosition().binlog_pos << std::endl; } } From 286436871660333dc4dfb309393f166813431c8a Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Sat, 9 May 2020 16:20:02 +0800 Subject: [PATCH 044/374] Fix conflicting file --- src/Core/tests/CMakeLists.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Core/tests/CMakeLists.txt b/src/Core/tests/CMakeLists.txt index f77163a7109..b24326d94b1 100644 --- a/src/Core/tests/CMakeLists.txt +++ b/src/Core/tests/CMakeLists.txt @@ -24,3 +24,8 @@ target_link_libraries (mysql_protocol PRIVATE dbms) if(USE_SSL) target_include_directories (mysql_protocol SYSTEM PRIVATE ${OPENSSL_INCLUDE_DIR}) endif() + +if (ENABLE_FUZZING) + add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp) + target_link_libraries (names_and_types_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) +endif () From b0657236f6f6716d14f589789e2756e9be4a5a95 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Sun, 10 May 2020 08:56:13 +0800 Subject: [PATCH 045/374] Add Null type value parse --- src/Core/MySQLReplication.cpp | 45 ++++++++++++++++++++++++----------- src/Core/MySQLReplication.h | 6 +++-- 2 files changed, 35 insertions(+), 16 deletions(-) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 1b7eeb7d53b..c45678bf020 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -26,9 +26,8 @@ namespace MySQLReplication return "BINLOG_CHECKSUM_ALG_ENUM_END"; case BINLOG_CHECKSUM_ALG_UNDEF: return "BINLOG_CHECKSUM_ALG_UNDEF"; - default: - return std::string("Unknown checksum alg: ") + std::to_string(static_cast(type)); } + return std::string("Unknown checksum alg: ") + std::to_string(static_cast(type)); } String ToString(EventType type) @@ -162,9 +161,6 @@ namespace MySQLReplication size_t len = header.event_size - (2 + 50 + 4 + 1 + EVENT_HEADER_LENGTH) - 1; event_type_header_length.resize(len); payload.readStrict(reinterpret_cast(event_type_header_length.data()), len); - assert(event_type_header_length[WRITE_ROWS_EVENT_V2] == ROWS_HEADER_LEN_V2); - assert(event_type_header_length[UPDATE_ROWS_EVENT_V2] == ROWS_HEADER_LEN_V2); - assert(event_type_header_length[DELETE_ROWS_EVENT_V2] == ROWS_HEADER_LEN_V2); } void FormatDescriptionEvent::print() const @@ -358,18 +354,24 @@ namespace MySQLReplication number_columns = readLengthEncodedNumber(payload); size_t columns_bitmap_size = (number_columns + 8) / 7; - payload.readStrict(reinterpret_cast(columns_before_bitmap.data()), columns_bitmap_size); - if (header.type == UPDATE_ROWS_EVENT_V2) + switch (header.type) { - payload.readStrict(reinterpret_cast(columns_after_bitmap.data()), columns_bitmap_size); + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + payload.readStrict(reinterpret_cast(columns_present_bitmap1.data()), columns_bitmap_size); + payload.readStrict(reinterpret_cast(columns_present_bitmap2.data()), columns_bitmap_size); + break; + default: + payload.readStrict(reinterpret_cast(columns_present_bitmap1.data()), columns_bitmap_size); + break; } while (payload.available() > CHECKSUM_CRC32_SIGNATURE_LENGTH) { - parseRow(payload, columns_before_bitmap); - if (header.type == UPDATE_ROWS_EVENT_V2) + parseRow(payload, columns_present_bitmap1); + if (header.type == UPDATE_ROWS_EVENT_V1 || header.type == UPDATE_ROWS_EVENT_V2) { - parseRow(payload, columns_after_bitmap); + parseRow(payload, columns_present_bitmap2); } } } @@ -379,18 +381,30 @@ namespace MySQLReplication UInt32 field_type = 0; UInt32 field_len = 0; - bitmap = ""; size_t columns_null_bitmap_size = (number_columns + 8) / 7; String columns_null_bitmap; columns_null_bitmap.resize(columns_null_bitmap_size); payload.readStrict(reinterpret_cast(columns_null_bitmap.data()), columns_null_bitmap_size); + Tuple row; for (auto i = 0U; i < number_columns; i++) { - field_type = table_map->column_type[i]; - UInt16 meta = table_map->column_meta[i]; + /// Column not presents. + if (!check_string_bit(bitmap, i)) + { + continue; + } + /// NULL column. + if (check_string_bit(columns_null_bitmap, i)) + { + row.push_back(Field{Null{}}); + continue; + } + + field_type = table_map->column_type[i]; + auto meta = table_map->column_meta[i]; if (field_type == MYSQL_TYPE_STRING) { if (meta >= 256) @@ -559,18 +573,21 @@ namespace MySQLReplication position.updateLogPos(event->header.log_pos); break; } + case WRITE_ROWS_EVENT_V1: case WRITE_ROWS_EVENT_V2: { event = std::make_shared(table_map); event->parseHeader(payload); event->parseEvent(payload); break; } + case DELETE_ROWS_EVENT_V1: case DELETE_ROWS_EVENT_V2: { event = std::make_shared(table_map); event->parseHeader(payload); event->parseEvent(payload); break; } + case UPDATE_ROWS_EVENT_V1: case UPDATE_ROWS_EVENT_V2: { event = std::make_shared(table_map); event->parseHeader(payload); diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index f41a946f631..a10d3779ada 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -23,6 +23,8 @@ namespace MySQLReplication class EventBase; using BinlogEventPtr = std::shared_ptr; + inline bool check_string_bit(String s, int k) { return (s[(k / 8)] & (1 << (k % 8))) != 0; } + enum BinlogChecksumAlg { BINLOG_CHECKSUM_ALG_OFF = 0, @@ -210,8 +212,8 @@ namespace MySQLReplication UInt32 number_columns; String schema; String table; - String columns_before_bitmap; - String columns_after_bitmap; + String columns_present_bitmap1; + String columns_present_bitmap2; std::vector rows; RowsEvent(std::shared_ptr table_map_) : table_map(table_map_) From 31818388be68842098f44a0fbe9b61bcae6017fc Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Sun, 10 May 2020 14:03:04 +0800 Subject: [PATCH 046/374] Support Int8/Int16/Int24 type --- src/Core/MySQLClient.h | 2 +- src/Core/MySQLReplication.cpp | 28 ++++++++++++++++++++++++++-- 2 files changed, 27 insertions(+), 3 deletions(-) diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 58caf1d91e2..55d4c834899 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -34,7 +34,7 @@ public: String error(); bool startBinlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos); BinlogEventPtr readOneBinlogEvent() ; - Position getPosition() const { return replication.getPosition(); }; + Position getPosition() const { return replication.getPosition(); } private: String host; diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index c45678bf020..a55e4e6385e 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -439,10 +439,34 @@ namespace MySQLReplication switch (field_type) { + case MYSQL_TYPE_TINY: { + Int8 val = 0; + payload.readStrict(reinterpret_cast(&val), 1); + row.push_back(Field{Int8{val}}); + break; + } + case MYSQL_TYPE_SHORT: { + Int16 val = 0; + payload.readStrict(reinterpret_cast(&val), 2); + row.push_back(Field{Int16{val}}); + break; + } + case MYSQL_TYPE_INT24: { + Int32 val = 0; + payload.readStrict(reinterpret_cast(&val), 3); + row.push_back(Field{Int32{val}}); + break; + } case MYSQL_TYPE_LONG: { - UInt32 val = 0; + Int32 val = 0; payload.readStrict(reinterpret_cast(&val), 4); - row.push_back(Field{UInt32{val}}); + row.push_back(Field{Int32{val}}); + break; + } + case MYSQL_TYPE_LONGLONG: { + Int64 val = 0; + payload.readStrict(reinterpret_cast(&val), 8); + row.push_back(Field{Int64{val}}); break; } case MYSQL_TYPE_VARCHAR: From a36ef50f92b9396d9c102126a49132ffaed6e199 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Sun, 10 May 2020 17:57:25 +0800 Subject: [PATCH 047/374] Support more types --- src/Core/MySQLReplication.cpp | 83 ++++++++++++++++++++++++++++------- 1 file changed, 67 insertions(+), 16 deletions(-) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index a55e4e6385e..15880f75810 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -358,10 +358,13 @@ namespace MySQLReplication { case UPDATE_ROWS_EVENT_V1: case UPDATE_ROWS_EVENT_V2: + columns_present_bitmap1.resize(columns_bitmap_size); + columns_present_bitmap2.resize(columns_bitmap_size); payload.readStrict(reinterpret_cast(columns_present_bitmap1.data()), columns_bitmap_size); payload.readStrict(reinterpret_cast(columns_present_bitmap2.data()), columns_bitmap_size); break; default: + columns_present_bitmap1.resize(columns_bitmap_size); payload.readStrict(reinterpret_cast(columns_present_bitmap1.data()), columns_bitmap_size); break; } @@ -469,20 +472,34 @@ namespace MySQLReplication row.push_back(Field{Int64{val}}); break; } + case MYSQL_TYPE_FLOAT: { + Float32 val = 0; + payload.readStrict(reinterpret_cast(&val), 4); + row.push_back(Field{Float32{val}}); + break; + } + case MYSQL_TYPE_DOUBLE: { + Float64 val = 0; + payload.readStrict(reinterpret_cast(&val), 8); + row.push_back(Field{Float64{val}}); + break; + } + case MYSQL_TYPE_TIMESTAMP: { + UInt32 val = 0; + payload.readStrict(reinterpret_cast(&val), 4); + row.push_back(Field{UInt32{val}}); + break; + } case MYSQL_TYPE_VARCHAR: case MYSQL_TYPE_VAR_STRING: { - uint32_t size = meta; - if (size < 256) + uint32_t size = 0; + if (meta < 256) { - uint8_t tmp1 = 0; - payload.readStrict(reinterpret_cast(&tmp1), 1); - size = tmp1; + payload.readStrict(reinterpret_cast(&size), 1); } else { - uint16_t tmp2 = 0; - payload.readStrict(reinterpret_cast(&tmp2), 2); - size = tmp2; + payload.readStrict(reinterpret_cast(&size), 2); } String val; @@ -492,21 +509,55 @@ namespace MySQLReplication break; } case MYSQL_TYPE_STRING: { - UInt32 size = field_len; - if (size < 256) + UInt32 size = 0; + if (field_len < 256) { - uint8_t tmp1 = 0; - payload.readStrict(reinterpret_cast(&tmp1), 1); - size = tmp1; + payload.readStrict(reinterpret_cast(&size), 1); } else { - uint16_t tmp2 = 0; - payload.readStrict(reinterpret_cast(&tmp2), 2); - size = tmp2; + payload.readStrict(reinterpret_cast(&size), 2); } + + String val; + val.resize(size); + payload.readStrict(reinterpret_cast(val.data()), size); + row.push_back(Field{String{val}}); break; } + case MYSQL_TYPE_GEOMETRY: + case MYSQL_TYPE_BLOB: { + UInt32 size = 0; + switch (meta) + { + case 1: { + payload.readStrict(reinterpret_cast(&size), 1); + break; + } + case 2: { + payload.readStrict(reinterpret_cast(&size), 2); + break; + } + case 3: { + payload.readStrict(reinterpret_cast(&size), 3); + break; + } + case 4: { + payload.readStrict(reinterpret_cast(&size), 4); + break; + } + default: + break; + } + + String val; + val.resize(size); + payload.readStrict(reinterpret_cast(val.data()), size); + row.push_back(Field{String{val}}); + break; + } + default: + throw ReplicationError("ParseRow: Unhandled field type:" + std::to_string(field_type), ErrorCodes::UNKNOWN_EXCEPTION); } } rows.push_back(row); From 81998cb1afc0f616ac34bd76ecb990929826dafe Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 11 May 2020 12:15:06 +0800 Subject: [PATCH 048/374] Support DATETIME/TIMESTAMP type --- src/Core/MySQLClient.cpp | 13 +- src/Core/MySQLClient.h | 5 +- src/Core/MySQLReplication.cpp | 591 ++++++++++++++++-------------- src/Core/MySQLReplication.h | 237 ++++++++++-- src/Core/tests/mysql_protocol.cpp | 87 +++-- 5 files changed, 610 insertions(+), 323 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index b7166701e99..8cbe645f4f9 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -126,21 +126,30 @@ bool MySQLClient::ping() return writeCommand(Command::COM_PING, ""); } -bool MySQLClient::startBinlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos) +bool MySQLClient::startBinlogDump(UInt32 slave_id, String replicate_db, String binlog_file_name, UInt64 binlog_pos) { + /// Set binlog checksum to CRC32. String checksum = "CRC32"; if (!writeCommand(Command::COM_QUERY, "SET @master_binlog_checksum = '" + checksum + "'")) { return false; } - /// 30s. + /// Set heartbeat 30s. UInt64 period_ns = (30 * 1e9); if (!writeCommand(Command::COM_QUERY, "SET @master_heartbeat_period = " + std::to_string(period_ns))) { return false; } + /// Set replication filter to master + /// This requires MySQL version >=5.6, so results are not checked here. + writeCommand(Command::COM_QUERY, "CHANGE REPLICATION FILTER REPLICATE_DO_DB = (" + replicate_db + ")"); + + /// Set Filter rule to replication. + replication.setReplicateDatabase(replicate_db); + + // Register slave. if (!registerSlaveOnMaster(slave_id)) { return false; diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 55d4c834899..06ec81e6adb 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -32,8 +32,9 @@ public: void disconnect(); bool ping(); String error(); - bool startBinlogDump(UInt32 slave_id, String binlog_file_name, UInt64 binlog_pos); - BinlogEventPtr readOneBinlogEvent() ; + + bool startBinlogDump(UInt32 slave_id, String replicate_db, String binlog_file_name, UInt64 binlog_pos); + BinlogEventPtr readOneBinlogEvent(); Position getPosition() const { return replication.getPosition(); } private: diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 15880f75810..7d87443c0bb 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -14,118 +14,6 @@ namespace MySQLReplication { using namespace MySQLProtocol; - String ToString(BinlogChecksumAlg type) - { - switch (type) - { - case BINLOG_CHECKSUM_ALG_OFF: - return "BINLOG_CHECKSUM_ALG_OFF"; - case BINLOG_CHECKSUM_ALG_CRC32: - return "BINLOG_CHECKSUM_ALG_CRC32"; - case BINLOG_CHECKSUM_ALG_ENUM_END: - return "BINLOG_CHECKSUM_ALG_ENUM_END"; - case BINLOG_CHECKSUM_ALG_UNDEF: - return "BINLOG_CHECKSUM_ALG_UNDEF"; - } - return std::string("Unknown checksum alg: ") + std::to_string(static_cast(type)); - } - - String ToString(EventType type) - { - switch (type) - { - case START_EVENT_V3: - return "StartEventV3"; - case QUERY_EVENT: - return "QueryEvent"; - case STOP_EVENT: - return "StopEvent"; - case ROTATE_EVENT: - return "RotateEvent"; - case INT_VAR_EVENT: - return "IntVarEvent"; - case LOAD_EVENT: - return "LoadEvent"; - case SLAVE_EVENT: - return "SlaveEvent"; - case CREATE_FILE_EVENT: - return "CreateFileEvent"; - case APPEND_BLOCK_EVENT: - return "AppendBlockEvent"; - case EXEC_LOAD_EVENT: - return "ExecLoadEvent"; - case DELETE_FILE_EVENT: - return "DeleteFileEvent"; - case NEW_LOAD_EVENT: - return "NewLoadEvent"; - case RAND_EVENT: - return "RandEvent"; - case USER_VAR_EVENT: - return "UserVarEvent"; - case FORMAT_DESCRIPTION_EVENT: - return "FormatDescriptionEvent"; - case XID_EVENT: - return "XIDEvent"; - case BEGIN_LOAD_QUERY_EVENT: - return "BeginLoadQueryEvent"; - case EXECUTE_LOAD_QUERY_EVENT: - return "ExecuteLoadQueryEvent"; - case TABLE_MAP_EVENT: - return "TableMapEvent"; - case WRITE_ROWS_EVENT_V0: - return "WriteRowsEventV0"; - case UPDATE_ROWS_EVENT_V0: - return "UpdateRowsEventV0"; - case DELETE_ROWS_EVENT_V0: - return "DeleteRowsEventV0"; - case WRITE_ROWS_EVENT_V1: - return "WriteRowsEventV1"; - case UPDATE_ROWS_EVENT_V1: - return "UpdateRowsEventV1"; - case DELETE_ROWS_EVENT_V1: - return "DeleteRowsEventV1"; - case INCIDENT_EVENT: - return "IncidentEvent"; - case HEARTBEAT_EVENT: - return "HeartbeatEvent"; - case IGNORABLE_EVENT: - return "IgnorableEvent"; - case ROWS_QUERY_EVENT: - return "RowsQueryEvent"; - case WRITE_ROWS_EVENT_V2: - return "WriteRowsEventV2"; - case UPDATE_ROWS_EVENT_V2: - return "UpdateRowsEventV2"; - case DELETE_ROWS_EVENT_V2: - return "DeleteRowsEventV2"; - case GTID_EVENT: - return "GTIDEvent"; - case ANONYMOUS_GTID_EVENT: - return "AnonymousGTIDEvent"; - case PREVIOUS_GTIDS_EVENT: - return "PreviousGTIDsEvent"; - case TRANSACTION_CONTEXT_EVENT: - return "TransactionContextEvent"; - case VIEW_CHANGE_EVENT: - return "ViewChangeEvent"; - case XA_PREPARE_LOG_EVENT: - return "XAPrepareLogEvent"; - case MARIA_ANNOTATE_ROWS_EVENT: - return "MariaAnnotateRowsEvent"; - case MARIA_BINLOG_CHECKPOINT_EVENT: - return "MariaBinlogCheckpointEvent"; - case MARIA_GTID_EVENT: - return "MariaGTIDEvent"; - case MARIA_GTID_LIST_EVENT: - return "MariaGTIDListEvent"; - case MARIA_START_ENCRYPTION_EVENT: - return "MariaStartEncryptionEvent"; - default: - break; - } - return std::string("Unknown event: ") + std::to_string(static_cast(type)); - } - /// https://dev.mysql.com/doc/internals/en/binlog-event-header.html void EventHeader::parse(ReadBuffer & payload) { @@ -139,7 +27,7 @@ namespace MySQLReplication void EventHeader::print() const { - std::cerr << "\n=== " << ToString(this->type) << " ===" << std::endl; + std::cerr << "\n=== " << to_string(this->type) << " ===" << std::endl; std::cerr << "Timestamp: " << this->timestamp << std::endl; std::cerr << "Event Type: " << this->type << std::endl; std::cerr << "Server ID: " << this->server_id << std::endl; @@ -353,19 +241,16 @@ namespace MySQLReplication payload.ignore(extra_data_len - 2); number_columns = readLengthEncodedNumber(payload); - size_t columns_bitmap_size = (number_columns + 8) / 7; + size_t columns_bitmap_size = (number_columns + 7) / 8; switch (header.type) { case UPDATE_ROWS_EVENT_V1: case UPDATE_ROWS_EVENT_V2: - columns_present_bitmap1.resize(columns_bitmap_size); - columns_present_bitmap2.resize(columns_bitmap_size); - payload.readStrict(reinterpret_cast(columns_present_bitmap1.data()), columns_bitmap_size); - payload.readStrict(reinterpret_cast(columns_present_bitmap2.data()), columns_bitmap_size); + readBitmap(payload, columns_present_bitmap1, columns_bitmap_size); + readBitmap(payload, columns_present_bitmap2, columns_bitmap_size); break; default: - columns_present_bitmap1.resize(columns_bitmap_size); - payload.readStrict(reinterpret_cast(columns_present_bitmap1.data()), columns_bitmap_size); + readBitmap(payload, columns_present_bitmap1, columns_bitmap_size); break; } @@ -379,186 +264,339 @@ namespace MySQLReplication } } - void RowsEvent::parseRow(ReadBuffer & payload, String bitmap) + void RowsEvent::parseRow(ReadBuffer & payload, Bitmap & bitmap) { + Tuple row; UInt32 field_type = 0; UInt32 field_len = 0; + UInt32 null_index = 0; - size_t columns_null_bitmap_size = (number_columns + 8) / 7; - String columns_null_bitmap; - columns_null_bitmap.resize(columns_null_bitmap_size); - payload.readStrict(reinterpret_cast(columns_null_bitmap.data()), columns_null_bitmap_size); + UInt32 re_count = 0; + for (auto i = 0U; i < number_columns; i++) + { + if (bitmap[i]) + re_count++; + } + re_count = (re_count + 7) / 8; + boost::dynamic_bitset<> columns_null_set; + readBitmap(payload, columns_null_set, re_count); - - Tuple row; for (auto i = 0U; i < number_columns; i++) { /// Column not presents. - if (!check_string_bit(bitmap, i)) - { + if (!bitmap[i]) continue; - } - /// NULL column. - if (check_string_bit(columns_null_bitmap, i)) + if (columns_null_set[null_index]) { row.push_back(Field{Null{}}); - continue; } - - field_type = table_map->column_type[i]; - auto meta = table_map->column_meta[i]; - if (field_type == MYSQL_TYPE_STRING) + else { - if (meta >= 256) + field_type = table_map->column_type[i]; + auto meta = table_map->column_meta[i]; + if (field_type == MYSQL_TYPE_STRING) { - UInt32 byte0 = meta >> 8; - UInt32 byte1 = meta & 0xff; - if ((byte0 & 0x30) != 0x30) + if (meta >= 256) { - field_len = byte1 | (((byte0 & 0x30) ^ 0x30) << 4); - field_type = byte0 | 0x30; - } - else - { - switch (byte0) + UInt32 byte0 = meta >> 8; + UInt32 byte1 = meta & 0xff; + if ((byte0 & 0x30) != 0x30) { - case MYSQL_TYPE_SET: - case MYSQL_TYPE_ENUM: - case MYSQL_TYPE_STRING: - field_type = byte0; - field_len = byte1; - break; - default: - throw ReplicationError("ParseRow: Illegal event", ErrorCodes::UNKNOWN_EXCEPTION); + field_len = byte1 | (((byte0 & 0x30) ^ 0x30) << 4); + field_type = byte0 | 0x30; + } + else + { + switch (byte0) + { + case MYSQL_TYPE_SET: + case MYSQL_TYPE_ENUM: + case MYSQL_TYPE_STRING: + field_type = byte0; + field_len = byte1; + break; + default: + throw ReplicationError("ParseRow: Unhandled binlog event", ErrorCodes::UNKNOWN_EXCEPTION); + } } } + else + { + field_len = meta; + } } - else + + switch (field_type) { - field_len = meta; - } - } + case MYSQL_TYPE_TINY: { + UInt8 val = 0; + payload.readStrict(reinterpret_cast(&val), 1); + row.push_back(Field{UInt8{val}}); + break; + } + case MYSQL_TYPE_SHORT: { + UInt16 val = 0; + payload.readStrict(reinterpret_cast(&val), 2); + row.push_back(Field{UInt16{val}}); + break; + } + case MYSQL_TYPE_INT24: { + Int32 val = 0; + payload.readStrict(reinterpret_cast(&val), 3); + row.push_back(Field{Int32{val}}); + break; + } + case MYSQL_TYPE_LONG: { + UInt32 val = 0; + payload.readStrict(reinterpret_cast(&val), 4); + row.push_back(Field{UInt32{val}}); + break; + } + case MYSQL_TYPE_LONGLONG: { + UInt64 val = 0; + payload.readStrict(reinterpret_cast(&val), 8); + row.push_back(Field{UInt64{val}}); + break; + } + case MYSQL_TYPE_FLOAT: { + Float64 val = 0; + payload.readStrict(reinterpret_cast(&val), 4); + row.push_back(Field{Float64{val}}); + break; + } + case MYSQL_TYPE_DOUBLE: { + Float64 val = 0; + payload.readStrict(reinterpret_cast(&val), 8); + row.push_back(Field{Float64{val}}); + break; + } + case MYSQL_TYPE_TIMESTAMP: { + UInt32 val = 0; + payload.readStrict(reinterpret_cast(&val), 4); - switch (field_type) - { - case MYSQL_TYPE_TINY: { - Int8 val = 0; - payload.readStrict(reinterpret_cast(&val), 1); - row.push_back(Field{Int8{val}}); - break; - } - case MYSQL_TYPE_SHORT: { - Int16 val = 0; - payload.readStrict(reinterpret_cast(&val), 2); - row.push_back(Field{Int16{val}}); - break; - } - case MYSQL_TYPE_INT24: { - Int32 val = 0; - payload.readStrict(reinterpret_cast(&val), 3); - row.push_back(Field{Int32{val}}); - break; - } - case MYSQL_TYPE_LONG: { - Int32 val = 0; - payload.readStrict(reinterpret_cast(&val), 4); - row.push_back(Field{Int32{val}}); - break; - } - case MYSQL_TYPE_LONGLONG: { - Int64 val = 0; - payload.readStrict(reinterpret_cast(&val), 8); - row.push_back(Field{Int64{val}}); - break; - } - case MYSQL_TYPE_FLOAT: { - Float32 val = 0; - payload.readStrict(reinterpret_cast(&val), 4); - row.push_back(Field{Float32{val}}); - break; - } - case MYSQL_TYPE_DOUBLE: { - Float64 val = 0; - payload.readStrict(reinterpret_cast(&val), 8); - row.push_back(Field{Float64{val}}); - break; - } - case MYSQL_TYPE_TIMESTAMP: { - UInt32 val = 0; - payload.readStrict(reinterpret_cast(&val), 4); - row.push_back(Field{UInt32{val}}); - break; - } - case MYSQL_TYPE_VARCHAR: - case MYSQL_TYPE_VAR_STRING: { - uint32_t size = 0; - if (meta < 256) - { - payload.readStrict(reinterpret_cast(&size), 1); - } - else - { - payload.readStrict(reinterpret_cast(&size), 2); + time_t time = time_t(val); + std::tm * gtm = std::gmtime(&time); + char buffer[32]; + std::strftime(buffer, 32, "%Y-%m-%d %H:%M:%S", gtm); + row.push_back(Field{String{buffer}}); + break; } + case MYSQL_TYPE_TIME: { + UInt32 i24 = 0; + payload.readStrict(reinterpret_cast(&i24), 3); - String val; - val.resize(size); - payload.readStrict(reinterpret_cast(val.data()), size); - row.push_back(Field{String{val}}); - break; - } - case MYSQL_TYPE_STRING: { - UInt32 size = 0; - if (field_len < 256) - { - payload.readStrict(reinterpret_cast(&size), 1); - } - else - { - payload.readStrict(reinterpret_cast(&size), 2); + String time_buff; + time_buff.resize(8); + sprintf( + time_buff.data(), + "%02d:%02d:%02d", + static_cast(i24 / 10000), + static_cast(i24 % 10000) / 100, + static_cast(i24 % 100)); + row.push_back(Field{String{time_buff}}); + break; } + case MYSQL_TYPE_DATE: { + UInt32 i24 = 0; + payload.readStrict(reinterpret_cast(&i24), 3); - String val; - val.resize(size); - payload.readStrict(reinterpret_cast(val.data()), size); - row.push_back(Field{String{val}}); - break; - } - case MYSQL_TYPE_GEOMETRY: - case MYSQL_TYPE_BLOB: { - UInt32 size = 0; - switch (meta) - { - case 1: { + String time_buff; + time_buff.resize(10); + sprintf( + time_buff.data(), + "%04d-%02d-%02d", + static_cast((i24 >> 9) & 0x7fff), + static_cast((i24 >> 5) & 0xf), + static_cast(i24 & 0x1f)); + row.push_back(Field{String{time_buff}}); + break; + } + case MYSQL_TYPE_YEAR: { + Int32 val = 0; + payload.readStrict(reinterpret_cast(&val), 1); + row.push_back(Field{Int32{val + 1900}}); + break; + } + case MYSQL_TYPE_TIME2: { + UInt32 val = 0, frac_part = 0; + char sign = 0x22; + + readBigEndianStrict(payload, reinterpret_cast(&val), 3); + if (readBits(val, 0, 1, 24) == 0) + { + sign = '-'; + val = ~val + 1; + } + UInt32 hour = readBits(val, 2, 10, 24); + UInt32 minute = readBits(val, 12, 6, 24); + UInt32 second = readBits(val, 18, 6, 24); + readTimeFractionalPart(payload, reinterpret_cast(&frac_part), meta); + + if (frac_part != 0) + { + String time_buff; + time_buff.resize(16); + sprintf( + time_buff.data(), + "%c%02d:%02d:%02d.%06d", + static_cast(sign), + static_cast(hour), + static_cast(minute), + static_cast(second), + static_cast(frac_part)); + row.push_back(Field{String{time_buff}}); + } + else + { + String time_buff; + time_buff.resize(9); + sprintf( + time_buff.data(), + "%c%02d:%02d:%02d", + static_cast(sign), + static_cast(hour), + static_cast(minute), + static_cast(second)); + row.push_back(Field{String{time_buff}}); + } + break; + } + case MYSQL_TYPE_DATETIME2: { + Int64 val = 0, fsp = 0; + readBigEndianStrict(payload, reinterpret_cast(&val), 5); + readTimeFractionalPart(payload, reinterpret_cast(&fsp), meta); + + struct tm timeinfo; + UInt32 year_month = readBits(val, 1, 17, 40); + timeinfo.tm_year = year_month / 13 - 1900; + timeinfo.tm_mon = year_month % 13; + timeinfo.tm_mday = readBits(val, 18, 5, 40); + timeinfo.tm_hour = readBits(val, 23, 5, 40); + timeinfo.tm_min = readBits(val, 28, 6, 40); + timeinfo.tm_sec = readBits(val, 34, 6, 40); + + time_t time = mktime(&timeinfo); + std::tm * gtm = std::gmtime(&time); + char buffer[32]; + std::strftime(buffer, 32, "%Y-%m-%d %H:%M:%S", gtm); + row.push_back(Field{String{buffer}}); + break; + } + case MYSQL_TYPE_TIMESTAMP2: { + UInt32 sec = 0, subsec = 0, whole_part = 0; + readBigEndianStrict(payload, reinterpret_cast(&sec), 4); + readTimeFractionalPart(payload, reinterpret_cast(&sec), meta); + + whole_part = (sec + subsec / 1e6); + time_t time = time_t(whole_part); + std::tm * gtm = std::gmtime(&time); + char buffer[32]; + std::strftime(buffer, 32, "%Y-%m-%d %H:%M:%S", gtm); + row.push_back(Field{String{buffer}}); + break; + } + case MYSQL_TYPE_ENUM: { + Int32 val = 0; + Int32 len = (meta & 0xff); + switch (len) + { + case 1: { + payload.readStrict(reinterpret_cast(&val), 1); + break; + } + case 2: { + payload.readStrict(reinterpret_cast(&val), 2); + break; + } + default: + break; + } + row.push_back(Field{Int32{val}}); + break; + } + case MYSQL_TYPE_VARCHAR: + case MYSQL_TYPE_VAR_STRING: { + uint32_t size = 0; + if (meta < 256) + { payload.readStrict(reinterpret_cast(&size), 1); - break; } - case 2: { + else + { payload.readStrict(reinterpret_cast(&size), 2); - break; } - case 3: { - payload.readStrict(reinterpret_cast(&size), 3); - break; - } - case 4: { - payload.readStrict(reinterpret_cast(&size), 4); - break; - } - default: - break; - } - String val; - val.resize(size); - payload.readStrict(reinterpret_cast(val.data()), size); - row.push_back(Field{String{val}}); - break; + String val; + val.resize(size); + payload.readStrict(reinterpret_cast(val.data()), size); + row.push_back(Field{String{val}}); + break; + } + case MYSQL_TYPE_STRING: { + UInt32 size = 0; + if (field_len < 256) + { + payload.readStrict(reinterpret_cast(&size), 1); + } + else + { + payload.readStrict(reinterpret_cast(&size), 2); + } + + String val; + val.resize(size); + payload.readStrict(reinterpret_cast(val.data()), size); + row.push_back(Field{String{val}}); + break; + } + case MYSQL_TYPE_GEOMETRY: + case MYSQL_TYPE_BLOB: { + UInt32 size = 0; + switch (meta) + { + case 1: { + payload.readStrict(reinterpret_cast(&size), 1); + break; + } + case 2: { + payload.readStrict(reinterpret_cast(&size), 2); + break; + } + case 3: { + payload.readStrict(reinterpret_cast(&size), 3); + break; + } + case 4: { + payload.readStrict(reinterpret_cast(&size), 4); + break; + } + default: + break; + } + + String val; + val.resize(size); + payload.readStrict(reinterpret_cast(val.data()), size); + row.push_back(Field{String{val}}); + break; + } + case MYSQL_TYPE_JSON: { + UInt32 size = 0; + payload.readStrict(reinterpret_cast(&size), meta); + + String val; + val.resize(size); + payload.readStrict(reinterpret_cast(val.data()), size); + row.push_back(Field{String{val}}); + break; + } + default: + throw ReplicationError( + "ParseRow: Unhandled MySQL field type:" + std::to_string(field_type), ErrorCodes::UNKNOWN_EXCEPTION); } - default: - throw ReplicationError("ParseRow: Unhandled field type:" + std::to_string(field_type), ErrorCodes::UNKNOWN_EXCEPTION); } + null_index++; } rows.push_back(row); } @@ -589,6 +627,8 @@ namespace MySQLReplication std::cerr << "[DryRun Event]" << std::endl; } + void MySQLFlavor::setReplicateDatabase(String db) { replicate_do_db = std::move(db); } + void MySQLFlavor::readPayloadImpl(ReadBuffer & payload) { UInt16 header = static_cast(*payload.position()); @@ -630,6 +670,8 @@ namespace MySQLReplication event->parseEvent(payload); if (event->header.event_size > QUERY_EVENT_BEGIN_LENGTH) position.updateLogPos(event->header.log_pos); + else + event = std::make_shared(); break; } case XID_EVENT: { @@ -650,21 +692,30 @@ namespace MySQLReplication } case WRITE_ROWS_EVENT_V1: case WRITE_ROWS_EVENT_V2: { - event = std::make_shared(table_map); + if (do_replicate()) + event = std::make_shared(table_map); + else + event = std::make_shared(); event->parseHeader(payload); event->parseEvent(payload); break; } case DELETE_ROWS_EVENT_V1: case DELETE_ROWS_EVENT_V2: { - event = std::make_shared(table_map); + if (do_replicate()) + event = std::make_shared(table_map); + else + event = std::make_shared(); event->parseHeader(payload); event->parseEvent(payload); break; } case UPDATE_ROWS_EVENT_V1: case UPDATE_ROWS_EVENT_V2: { - event = std::make_shared(table_map); + if (do_replicate()) + event = std::make_shared(table_map); + else + event = std::make_shared(); event->parseHeader(payload); event->parseEvent(payload); break; diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index a10d3779ada..1c33e2918a6 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -6,6 +6,7 @@ #include #include +#include /// Implementation of MySQL replication protocol. /// Works only on little-endian architecture. @@ -18,13 +19,80 @@ namespace MySQLReplication static const int EVENT_HEADER_LENGTH = 19; static const int CHECKSUM_CRC32_SIGNATURE_LENGTH = 4; static const int QUERY_EVENT_BEGIN_LENGTH = 74; - static const int ROWS_HEADER_LEN_V2 = 10; + + using Bitmap = boost::dynamic_bitset<>; + + inline UInt64 readBits(UInt64 val, UInt8 start, UInt8 size, UInt8 length) + { + val = val >> (length - (start + size)); + return val & (UInt64(1 << size) - 1); + } + + inline void readBigEndianStrict(ReadBuffer & payload, char * to, size_t n) + { + payload.readStrict(to, n); + char *start = to, *end = to + n; + std::reverse(start, end); + } + + inline void readTimeFractionalPart(ReadBuffer & payload, char * to, UInt16 meta) + { + switch (meta) + { + case 1: + case 2: { + readBigEndianStrict(payload, to, 1); + break; + } + case 3: + case 4: { + readBigEndianStrict(payload, to, 2); + break; + } + case 5: + case 6: { + readBigEndianStrict(payload, to, 3); + break; + } + default: + break; + } + } + + inline void readBitmap(ReadBuffer & payload, Bitmap & bitmap, size_t bitmap_size) + { + String byte_buffer; + byte_buffer.resize(bitmap_size); + payload.readStrict(reinterpret_cast(byte_buffer.data()), bitmap_size); + bitmap.resize(bitmap_size * 8, false); + for (size_t i = 0; i < bitmap_size; ++i) + { + uint8_t tmp = byte_buffer[i]; + boost::dynamic_bitset<>::size_type bit = i * 8; + if (tmp == 0) + continue; + if ((tmp & 0x01) != 0) + bitmap.set(bit); + if ((tmp & 0x02) != 0) + bitmap.set(bit + 1); + if ((tmp & 0x04) != 0) + bitmap.set(bit + 2); + if ((tmp & 0x08) != 0) + bitmap.set(bit + 3); + if ((tmp & 0x10) != 0) + bitmap.set(bit + 4); + if ((tmp & 0x20) != 0) + bitmap.set(bit + 5); + if ((tmp & 0x40) != 0) + bitmap.set(bit + 6); + if ((tmp & 0x80) != 0) + bitmap.set(bit + 7); + } + } class EventBase; using BinlogEventPtr = std::shared_ptr; - inline bool check_string_bit(String s, int k) { return (s[(k / 8)] & (1 << (k % 8))) != 0; } - enum BinlogChecksumAlg { BINLOG_CHECKSUM_ALG_OFF = 0, @@ -32,7 +100,22 @@ namespace MySQLReplication BINLOG_CHECKSUM_ALG_ENUM_END, BINLOG_CHECKSUM_ALG_UNDEF = 255 }; - String ToString(BinlogChecksumAlg type); + + inline String to_string(BinlogChecksumAlg type) + { + switch (type) + { + case BINLOG_CHECKSUM_ALG_OFF: + return "BINLOG_CHECKSUM_ALG_OFF"; + case BINLOG_CHECKSUM_ALG_CRC32: + return "BINLOG_CHECKSUM_ALG_CRC32"; + case BINLOG_CHECKSUM_ALG_ENUM_END: + return "BINLOG_CHECKSUM_ALG_ENUM_END"; + case BINLOG_CHECKSUM_ALG_UNDEF: + return "BINLOG_CHECKSUM_ALG_UNDEF"; + } + return std::string("Unknown checksum alg: ") + std::to_string(static_cast(type)); + } /// http://dev.mysql.com/doc/internals/en/binlog-event-type.html enum EventType @@ -84,7 +167,111 @@ namespace MySQLReplication MARIA_GTID_LIST_EVENT = 163, MARIA_START_ENCRYPTION_EVENT = 164, }; - String ToString(EventType type); + + inline String to_string(EventType type) + { + switch (type) + { + case START_EVENT_V3: + return "StartEventV3"; + case QUERY_EVENT: + return "QueryEvent"; + case STOP_EVENT: + return "StopEvent"; + case ROTATE_EVENT: + return "RotateEvent"; + case INT_VAR_EVENT: + return "IntVarEvent"; + case LOAD_EVENT: + return "LoadEvent"; + case SLAVE_EVENT: + return "SlaveEvent"; + case CREATE_FILE_EVENT: + return "CreateFileEvent"; + case APPEND_BLOCK_EVENT: + return "AppendBlockEvent"; + case EXEC_LOAD_EVENT: + return "ExecLoadEvent"; + case DELETE_FILE_EVENT: + return "DeleteFileEvent"; + case NEW_LOAD_EVENT: + return "NewLoadEvent"; + case RAND_EVENT: + return "RandEvent"; + case USER_VAR_EVENT: + return "UserVarEvent"; + case FORMAT_DESCRIPTION_EVENT: + return "FormatDescriptionEvent"; + case XID_EVENT: + return "XIDEvent"; + case BEGIN_LOAD_QUERY_EVENT: + return "BeginLoadQueryEvent"; + case EXECUTE_LOAD_QUERY_EVENT: + return "ExecuteLoadQueryEvent"; + case TABLE_MAP_EVENT: + return "TableMapEvent"; + case WRITE_ROWS_EVENT_V0: + return "WriteRowsEventV0"; + case UPDATE_ROWS_EVENT_V0: + return "UpdateRowsEventV0"; + case DELETE_ROWS_EVENT_V0: + return "DeleteRowsEventV0"; + case WRITE_ROWS_EVENT_V1: + return "WriteRowsEventV1"; + case UPDATE_ROWS_EVENT_V1: + return "UpdateRowsEventV1"; + case DELETE_ROWS_EVENT_V1: + return "DeleteRowsEventV1"; + case INCIDENT_EVENT: + return "IncidentEvent"; + case HEARTBEAT_EVENT: + return "HeartbeatEvent"; + case IGNORABLE_EVENT: + return "IgnorableEvent"; + case ROWS_QUERY_EVENT: + return "RowsQueryEvent"; + case WRITE_ROWS_EVENT_V2: + return "WriteRowsEventV2"; + case UPDATE_ROWS_EVENT_V2: + return "UpdateRowsEventV2"; + case DELETE_ROWS_EVENT_V2: + return "DeleteRowsEventV2"; + case GTID_EVENT: + return "GTIDEvent"; + case ANONYMOUS_GTID_EVENT: + return "AnonymousGTIDEvent"; + case PREVIOUS_GTIDS_EVENT: + return "PreviousGTIDsEvent"; + case TRANSACTION_CONTEXT_EVENT: + return "TransactionContextEvent"; + case VIEW_CHANGE_EVENT: + return "ViewChangeEvent"; + case XA_PREPARE_LOG_EVENT: + return "XAPrepareLogEvent"; + case MARIA_ANNOTATE_ROWS_EVENT: + return "MariaAnnotateRowsEvent"; + case MARIA_BINLOG_CHECKPOINT_EVENT: + return "MariaBinlogCheckpointEvent"; + case MARIA_GTID_EVENT: + return "MariaGTIDEvent"; + case MARIA_GTID_LIST_EVENT: + return "MariaGTIDListEvent"; + case MARIA_START_ENCRYPTION_EVENT: + return "MariaStartEncryptionEvent"; + default: + break; + } + return std::string("Unknown event: ") + std::to_string(static_cast(type)); + } + + enum MySQLEventType + { + MYSQL_UNHANDLED_EVENT = 0, + MYSQL_QUERY_EVENT = 1, + MYSQL_WRITE_ROWS_EVENT = 2, + MYSQL_UPDATE_ROWS_EVENT = 3, + MYSQL_DELETE_ROWS_EVENT = 4, + }; class ReplicationError : public DB::Exception { @@ -114,10 +301,8 @@ namespace MySQLReplication virtual ~EventBase() = default; virtual void print() const = 0; virtual void parseHeader(ReadBuffer & payload) { header.parse(payload); } - virtual void parseEvent(ReadBuffer & payload) { parseImpl(payload); } - - EventType type() const { return header.type; } + virtual MySQLEventType type() const { return MYSQL_UNHANDLED_EVENT; } protected: virtual void parseImpl(ReadBuffer & payload) = 0; @@ -125,7 +310,7 @@ namespace MySQLReplication class FormatDescriptionEvent : public EventBase { - public: + protected: UInt16 binlog_version; String server_version; UInt32 create_timestamp; @@ -133,8 +318,6 @@ namespace MySQLReplication String event_type_header_length; void print() const override; - - protected: void parseImpl(ReadBuffer & payload) override; private: @@ -166,6 +349,7 @@ namespace MySQLReplication String query; void print() const override; + MySQLEventType type() const override { return MYSQL_QUERY_EVENT; } protected: void parseImpl(ReadBuffer & payload) override; @@ -173,12 +357,10 @@ namespace MySQLReplication class XIDEvent : public EventBase { - public: + protected: UInt64 xid; void print() const override; - - protected: void parseImpl(ReadBuffer & payload) override; }; @@ -206,26 +388,29 @@ namespace MySQLReplication class RowsEvent : public EventBase { public: - UInt64 table_id; - UInt16 flags; - UInt16 extra_data_len; UInt32 number_columns; String schema; String table; - String columns_present_bitmap1; - String columns_present_bitmap2; std::vector rows; - RowsEvent(std::shared_ptr table_map_) : table_map(table_map_) + RowsEvent(std::shared_ptr table_map_) + : number_columns(0), table_id(0), flags(0), extra_data_len(0), table_map(table_map_) { schema = table_map->schema; table = table_map->table; } + void print() const override; protected: + UInt64 table_id; + UInt16 flags; + UInt16 extra_data_len; + Bitmap columns_present_bitmap1; + Bitmap columns_present_bitmap2; + void parseImpl(ReadBuffer & payload) override; - void parseRow(ReadBuffer & payload, String bitmap); + void parseRow(ReadBuffer & payload, Bitmap & bitmap); private: std::shared_ptr table_map; @@ -235,18 +420,21 @@ namespace MySQLReplication { public: WriteRowsEvent(std::shared_ptr table_map_) : RowsEvent(table_map_) { } + MySQLEventType type() const override { return MYSQL_WRITE_ROWS_EVENT; } }; class DeleteRowsEvent : public RowsEvent { public: DeleteRowsEvent(std::shared_ptr table_map_) : RowsEvent(table_map_) { } + MySQLEventType type() const override { return MYSQL_DELETE_ROWS_EVENT; } }; class UpdateRowsEvent : public RowsEvent { public: UpdateRowsEvent(std::shared_ptr table_map_) : RowsEvent(table_map_) { } + MySQLEventType type() const override { return MYSQL_UPDATE_ROWS_EVENT; } }; class DryRunEvent : public EventBase @@ -274,6 +462,7 @@ namespace MySQLReplication virtual String getName() const = 0; virtual Position getPosition() const = 0; virtual BinlogEventPtr readOneEvent() = 0; + virtual void setReplicateDatabase(String db) = 0; virtual ~IFlavor() = default; }; @@ -282,14 +471,18 @@ namespace MySQLReplication public: BinlogEventPtr event; + void readPayloadImpl(ReadBuffer & payload) override; + void setReplicateDatabase(String db) override; String getName() const override { return "MySQL"; } Position getPosition() const override { return position; } - void readPayloadImpl(ReadBuffer & payload) override; BinlogEventPtr readOneEvent() override { return event; } private: Position position; + String replicate_do_db; std::shared_ptr table_map; + + inline bool do_replicate() { return (replicate_do_db.empty() || (table_map->schema == replicate_do_db)); } }; } diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index 1c6b61b742e..7b081f50602 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -162,35 +162,68 @@ int main(int, char **) } { - UInt32 slave_id = 9004; - MySQLClient slave("127.0.0.1", 9001, "default", "123"); - if (!slave.connect()) + try { - std::cerr << "Connect Error: " << slave.error() << std::endl; + UInt32 slave_id = 9004; + MySQLClient slave("127.0.0.1", 9001, "default", "123"); + if (!slave.connect()) + { + std::cerr << "Connect Error: " << slave.error() << std::endl; + return 1; + } + + if (!slave.startBinlogDump(slave_id, "", "", 4)) + { + std::cerr << "Connect Error: " << slave.error() << std::endl; + return 1; + } + + while (true) + { + auto event = slave.readOneBinlogEvent(); + switch (event->type()) + { + case MYSQL_QUERY_EVENT: { + auto binlogEvent = std::dynamic_pointer_cast(event); + binlogEvent->print(); + + Position pos = slave.getPosition(); + std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; + break; + } + case MYSQL_WRITE_ROWS_EVENT: { + auto binlogEvent = std::dynamic_pointer_cast(event); + binlogEvent->print(); + + Position pos = slave.getPosition(); + std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; + break; + } + case MYSQL_UPDATE_ROWS_EVENT: { + auto binlogEvent = std::dynamic_pointer_cast(event); + binlogEvent->print(); + + Position pos = slave.getPosition(); + std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; + break; + } + case MYSQL_DELETE_ROWS_EVENT: { + auto binlogEvent = std::dynamic_pointer_cast(event); + binlogEvent->print(); + + Position pos = slave.getPosition(); + std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; + break; + } + default: + break; + } + } + } + catch (const Exception & ex) + { + std::cerr << "Error: " << ex.message() << std::endl; return 1; } - - if (!slave.ping()) - { - std::cerr << "Connect Error: " << slave.error() << std::endl; - return 1; - } - - if (!slave.startBinlogDump(slave_id, "", 4)) - { - std::cerr << "Connect Error: " << slave.error() << std::endl; - assert(0); - } - - while (true) - { - auto event = slave.readOneBinlogEvent(); - ASSERT(event != nullptr) - event->print(); - std::cerr << "Binlog Name: " << slave.getPosition().binlog_name << std::endl; - std::cerr << "Binlog Pos: " << slave.getPosition().binlog_pos << std::endl; - } } - - return 0; } From 5219a34b322e4230b6492d1f614638680bbcb7be Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 12 May 2020 13:49:53 +0800 Subject: [PATCH 049/374] Add QueryEvent type --- src/Core/MySQLReplication.cpp | 74 ++++++++++++++++++----------------- src/Core/MySQLReplication.h | 12 +++++- 2 files changed, 48 insertions(+), 38 deletions(-) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 7d87443c0bb..4aead1974f8 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -96,6 +96,15 @@ namespace MySQLReplication = header.event_size - EVENT_HEADER_LENGTH - 4 - 4 - 1 - 2 - 2 - status_len - schema_len - 1 - CHECKSUM_CRC32_SIGNATURE_LENGTH; query.resize(len); payload.readStrict(reinterpret_cast(query.data()), len); + + if (query == "BEGIN") + { + typ = BEGIN; + } + else if (query == "SAVEPOINT") + { + typ = SAVEPOINT; + } } void QueryEvent::print() const @@ -267,7 +276,6 @@ namespace MySQLReplication void RowsEvent::parseRow(ReadBuffer & payload, Bitmap & bitmap) { Tuple row; - UInt32 field_type = 0; UInt32 field_len = 0; UInt32 null_index = 0; @@ -293,8 +301,8 @@ namespace MySQLReplication } else { - field_type = table_map->column_type[i]; auto meta = table_map->column_meta[i]; + auto field_type = table_map->column_type[i]; if (field_type == MYSQL_TYPE_STRING) { if (meta >= 256) @@ -374,12 +382,7 @@ namespace MySQLReplication case MYSQL_TYPE_TIMESTAMP: { UInt32 val = 0; payload.readStrict(reinterpret_cast(&val), 4); - - time_t time = time_t(val); - std::tm * gtm = std::gmtime(&time); - char buffer[32]; - std::strftime(buffer, 32, "%Y-%m-%d %H:%M:%S", gtm); - row.push_back(Field{String{buffer}}); + row.push_back(Field{UInt64{val}}); break; } case MYSQL_TYPE_TIME: { @@ -415,17 +418,19 @@ namespace MySQLReplication case MYSQL_TYPE_YEAR: { Int32 val = 0; payload.readStrict(reinterpret_cast(&val), 1); - row.push_back(Field{Int32{val + 1900}}); + + String time_buff; + time_buff.resize(4); + sprintf(time_buff.data(), "%04d", (val + 1900)); + row.push_back(Field{String{time_buff}}); break; } case MYSQL_TYPE_TIME2: { UInt32 val = 0, frac_part = 0; - char sign = 0x22; readBigEndianStrict(payload, reinterpret_cast(&val), 3); if (readBits(val, 0, 1, 24) == 0) { - sign = '-'; val = ~val + 1; } UInt32 hour = readBits(val, 2, 10, 24); @@ -436,11 +441,10 @@ namespace MySQLReplication if (frac_part != 0) { String time_buff; - time_buff.resize(16); + time_buff.resize(15); sprintf( time_buff.data(), - "%c%02d:%02d:%02d.%06d", - static_cast(sign), + "%02d:%02d:%02d.%06d", static_cast(hour), static_cast(minute), static_cast(second), @@ -450,11 +454,10 @@ namespace MySQLReplication else { String time_buff; - time_buff.resize(9); + time_buff.resize(8); sprintf( time_buff.data(), - "%c%02d:%02d:%02d", - static_cast(sign), + "%02d:%02d:%02d", static_cast(hour), static_cast(minute), static_cast(second)); @@ -469,31 +472,22 @@ namespace MySQLReplication struct tm timeinfo; UInt32 year_month = readBits(val, 1, 17, 40); - timeinfo.tm_year = year_month / 13 - 1900; - timeinfo.tm_mon = year_month % 13; + timeinfo.tm_year = (year_month / 13) - 1900; + timeinfo.tm_mon = (year_month % 13) - 1; timeinfo.tm_mday = readBits(val, 18, 5, 40); timeinfo.tm_hour = readBits(val, 23, 5, 40); timeinfo.tm_min = readBits(val, 28, 6, 40); timeinfo.tm_sec = readBits(val, 34, 6, 40); time_t time = mktime(&timeinfo); - std::tm * gtm = std::gmtime(&time); - char buffer[32]; - std::strftime(buffer, 32, "%Y-%m-%d %H:%M:%S", gtm); - row.push_back(Field{String{buffer}}); + row.push_back(Field{UInt64{static_cast(time)}}); break; } case MYSQL_TYPE_TIMESTAMP2: { - UInt32 sec = 0, subsec = 0, whole_part = 0; + UInt64 sec = 0, fsp = 0; readBigEndianStrict(payload, reinterpret_cast(&sec), 4); - readTimeFractionalPart(payload, reinterpret_cast(&sec), meta); - - whole_part = (sec + subsec / 1e6); - time_t time = time_t(whole_part); - std::tm * gtm = std::gmtime(&time); - char buffer[32]; - std::strftime(buffer, 32, "%Y-%m-%d %H:%M:%S", gtm); - row.push_back(Field{String{buffer}}); + readTimeFractionalPart(payload, reinterpret_cast(&fsp), meta); + row.push_back(Field{UInt64{sec}}); break; } case MYSQL_TYPE_ENUM: { @@ -668,10 +662,18 @@ namespace MySQLReplication event = std::make_shared(); event->parseHeader(payload); event->parseEvent(payload); - if (event->header.event_size > QUERY_EVENT_BEGIN_LENGTH) - position.updateLogPos(event->header.log_pos); - else - event = std::make_shared(); + + auto query = std::dynamic_pointer_cast(event); + switch (query->typ) + { + case BEGIN: + case SAVEPOINT: { + event = std::make_shared(); + break; + } + default: + position.updateLogPos(event->header.log_pos); + } break; } case XID_EVENT: { diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index 1c33e2918a6..54d4a058658 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -18,14 +18,14 @@ namespace MySQLReplication static const int EVENT_VERSION_V4 = 4; static const int EVENT_HEADER_LENGTH = 19; static const int CHECKSUM_CRC32_SIGNATURE_LENGTH = 4; - static const int QUERY_EVENT_BEGIN_LENGTH = 74; using Bitmap = boost::dynamic_bitset<>; inline UInt64 readBits(UInt64 val, UInt8 start, UInt8 size, UInt8 length) { + UInt64 mask = 1; val = val >> (length - (start + size)); - return val & (UInt64(1 << size) - 1); + return val & ((mask << size) - 1); } inline void readBigEndianStrict(ReadBuffer & payload, char * to, size_t n) @@ -336,6 +336,13 @@ namespace MySQLReplication void parseImpl(ReadBuffer & payload) override; }; + enum QueryType + { + DDL = 0, + BEGIN = 1, + SAVEPOINT = 2 + }; + class QueryEvent : public EventBase { public: @@ -347,6 +354,7 @@ namespace MySQLReplication String status; String schema; String query; + QueryType typ = DDL; void print() const override; MySQLEventType type() const override { return MYSQL_QUERY_EVENT; } From 5c23583cdaf8f55ea5c6e0501af1c66a322d5f23 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 12 May 2020 21:22:55 +0800 Subject: [PATCH 050/374] Add DEMICAL type --- src/Core/MySQLReplication.cpp | 95 ++++++++++++++++++++++++++++++- src/Core/tests/mysql_protocol.cpp | 8 +-- 2 files changed, 96 insertions(+), 7 deletions(-) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 4aead1974f8..1914c96b3bf 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -1,6 +1,7 @@ #include "MySQLReplication.h" #include +#include #include namespace DB @@ -335,6 +336,12 @@ namespace MySQLReplication } } + /// Types that do not used in the binlog event: + /// MYSQL_TYPE_ENUM + /// MYSQL_TYPE_SET + /// MYSQL_TYPE_TINY_BLOB + /// MYSQL_TYPE_MEDIUM_BLOB + /// MYSQL_TYPE_LONG_BLOB switch (field_type) { case MYSQL_TYPE_TINY: { @@ -490,6 +497,88 @@ namespace MySQLReplication row.push_back(Field{UInt64{sec}}); break; } + case MYSQL_TYPE_NEWDECIMAL: { + Int8 digits_per_integer = 9; + Int8 precision = meta >> 8; + Int8 decimals = meta & 0xff; + const char compressed_byte_map[] = {0, 1, 1, 2, 2, 3, 3, 4, 4, 4}; + + Int8 integral = (precision - decimals); + UInt32 uncompressed_integers = integral / digits_per_integer; + UInt32 uncompressed_decimals = decimals / digits_per_integer; + UInt32 compressed_integers = integral - (uncompressed_integers * digits_per_integer); + UInt32 compressed_decimals = decimals - (uncompressed_decimals * digits_per_integer); + + String buff; + UInt32 bytes_to_read = uncompressed_integers * 4 + compressed_byte_map[compressed_integers] + + uncompressed_decimals * 4 + compressed_byte_map[compressed_decimals]; + buff.resize(bytes_to_read); + payload.readStrict(reinterpret_cast(buff.data()), bytes_to_read); + + String format; + format.resize(0); + + bool is_negative = ((buff[0] & 0x80) == 0); + if (is_negative) + { + format += "-"; + } + buff[0] ^= 0x80; + + ReadBufferFromString reader(buff); + /// Compressed part. + if (compressed_integers != 0) + { + Int64 val = 0; + UInt32 to_read = compressed_byte_map[compressed_integers]; + readBigEndianStrict(reader, reinterpret_cast(&val), to_read); + format += std::to_string(val); + } + + for (auto k = 0; k < uncompressed_integers; k++) + { + UInt32 val = 0; + readBigEndianStrict(reader, reinterpret_cast(&val), 4); + format += std::to_string(val); + } + format += "."; + for (auto k = 0; k < uncompressed_decimals; k++) + { + UInt32 val = 0; + reader.readStrict(reinterpret_cast(&val), 4); + format += std::to_string(val); + } + + /// Compressed part. + if (compressed_decimals != 0) + { + Int64 val = 0; + String compressed_buff; + UInt32 to_read = compressed_byte_map[compressed_decimals]; + switch (to_read) + { + case 1: { + reader.readStrict(reinterpret_cast(&val), 1); + break; + } + case 2: { + readBigEndianStrict(reader, reinterpret_cast(&val), 2); + break; + } + case 3: { + readBigEndianStrict(reader, reinterpret_cast(&val), 3); + break; + } + case 4: { + readBigEndianStrict(reader, reinterpret_cast(&val), 4); + break; + } + } + format += std::to_string(val); + } + row.push_back(Field{String{format}}); + break; + } case MYSQL_TYPE_ENUM: { Int32 val = 0; Int32 len = (meta & 0xff); @@ -653,7 +742,7 @@ namespace MySQLReplication event->parseHeader(payload); event->parseEvent(payload); - auto rotate = std::dynamic_pointer_cast(event); + auto rotate = std::static_pointer_cast(event); position.updateLogPos(event->header.log_pos); position.updateLogName(rotate->next_binlog); break; @@ -663,7 +752,7 @@ namespace MySQLReplication event->parseHeader(payload); event->parseEvent(payload); - auto query = std::dynamic_pointer_cast(event); + auto query = std::static_pointer_cast(event); switch (query->typ) { case BEGIN: @@ -688,7 +777,7 @@ namespace MySQLReplication event->parseHeader(payload); event->parseEvent(payload); - table_map = std::dynamic_pointer_cast(event); + table_map = std::static_pointer_cast(event); position.updateLogPos(event->header.log_pos); break; } diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index 7b081f50602..89cd07d260e 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -184,7 +184,7 @@ int main(int, char **) switch (event->type()) { case MYSQL_QUERY_EVENT: { - auto binlogEvent = std::dynamic_pointer_cast(event); + auto binlogEvent = std::static_pointer_cast(event); binlogEvent->print(); Position pos = slave.getPosition(); @@ -192,7 +192,7 @@ int main(int, char **) break; } case MYSQL_WRITE_ROWS_EVENT: { - auto binlogEvent = std::dynamic_pointer_cast(event); + auto binlogEvent = std::static_pointer_cast(event); binlogEvent->print(); Position pos = slave.getPosition(); @@ -200,7 +200,7 @@ int main(int, char **) break; } case MYSQL_UPDATE_ROWS_EVENT: { - auto binlogEvent = std::dynamic_pointer_cast(event); + auto binlogEvent = std::static_pointer_cast(event); binlogEvent->print(); Position pos = slave.getPosition(); @@ -208,7 +208,7 @@ int main(int, char **) break; } case MYSQL_DELETE_ROWS_EVENT: { - auto binlogEvent = std::dynamic_pointer_cast(event); + auto binlogEvent = std::static_pointer_cast(event); binlogEvent->print(); Position pos = slave.getPosition(); From 9f0e74ff93a6b027d2addfd3b70bb42ef877f6de Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 12 May 2020 21:40:10 +0800 Subject: [PATCH 051/374] Change the MySQL Client api from bool to throw execption --- src/Core/MySQLClient.cpp | 53 ++++++++----------------------- src/Core/MySQLClient.h | 14 ++++---- src/Core/tests/mysql_protocol.cpp | 16 ++++------ 3 files changed, 26 insertions(+), 57 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 8cbe645f4f9..ecf9e70c7e1 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -17,7 +17,7 @@ MySQLClient::MySQLClient(const String & host_, UInt16 port_, const String & user client_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_PLUGIN_AUTH | CLIENT_SECURE_CONNECTION; } -bool MySQLClient::connect() +void MySQLClient::connect() { if (connected) { @@ -39,7 +39,7 @@ bool MySQLClient::connect() in = std::make_shared(*socket); out = std::make_shared(*socket); packet_sender = std::make_shared(*in, *out, seq); - return handshake(); + handshake(); } void MySQLClient::disconnect() @@ -53,7 +53,7 @@ void MySQLClient::disconnect() } /// https://dev.mysql.com/doc/internals/en/connection-phase-packets.html -bool MySQLClient::handshake() +void MySQLClient::handshake() { Handshake handshake; packet_sender->receivePacket(handshake); @@ -76,15 +76,12 @@ bool MySQLClient::handshake() packet_sender->resetSequenceId(); if (packet_response.getType() == PACKET_ERR) { - last_error = packet_response.err.error_message; + throw MySQLClientError(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); } - return (packet_response.getType() != PACKET_ERR); } -bool MySQLClient::writeCommand(char command, String query) +void MySQLClient::writeCommand(char command, String query) { - bool ret = false; - WriteCommand write_command(command, query); packet_sender->sendPacket(write_command, true); @@ -93,19 +90,16 @@ bool MySQLClient::writeCommand(char command, String query) switch (packet_response.getType()) { case PACKET_ERR: - last_error = packet_response.err.error_message; - break; + throw MySQLClientError(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); case PACKET_OK: - ret = true; break; default: break; } packet_sender->resetSequenceId(); - return ret; } -bool MySQLClient::registerSlaveOnMaster(UInt32 slave_id) +void MySQLClient::registerSlaveOnMaster(UInt32 slave_id) { RegisterSlave register_slave(slave_id); packet_sender->sendPacket(register_slave, true); @@ -114,33 +108,23 @@ bool MySQLClient::registerSlaveOnMaster(UInt32 slave_id) packet_sender->receivePacket(packet_response); packet_sender->resetSequenceId(); if (packet_response.getType() == PACKET_ERR) - { - last_error = packet_response.err.error_message; - return false; - } - return true; + throw MySQLClientError(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); } -bool MySQLClient::ping() +void MySQLClient::ping() { - return writeCommand(Command::COM_PING, ""); + writeCommand(Command::COM_PING, ""); } -bool MySQLClient::startBinlogDump(UInt32 slave_id, String replicate_db, String binlog_file_name, UInt64 binlog_pos) +void MySQLClient::startBinlogDump(UInt32 slave_id, String replicate_db, String binlog_file_name, UInt64 binlog_pos) { /// Set binlog checksum to CRC32. String checksum = "CRC32"; - if (!writeCommand(Command::COM_QUERY, "SET @master_binlog_checksum = '" + checksum + "'")) - { - return false; - } + writeCommand(Command::COM_QUERY, "SET @master_binlog_checksum = '" + checksum + "'"); /// Set heartbeat 30s. UInt64 period_ns = (30 * 1e9); - if (!writeCommand(Command::COM_QUERY, "SET @master_heartbeat_period = " + std::to_string(period_ns))) - { - return false; - } + writeCommand(Command::COM_QUERY, "SET @master_heartbeat_period = " + std::to_string(period_ns)); /// Set replication filter to master /// This requires MySQL version >=5.6, so results are not checked here. @@ -150,15 +134,11 @@ bool MySQLClient::startBinlogDump(UInt32 slave_id, String replicate_db, String b replication.setReplicateDatabase(replicate_db); // Register slave. - if (!registerSlaveOnMaster(slave_id)) - { - return false; - } + registerSlaveOnMaster(slave_id); binlog_pos = binlog_pos < 4 ? 4 : binlog_pos; BinlogDump binlog_dump(binlog_pos, binlog_file_name, slave_id); packet_sender->sendPacket(binlog_dump, true); - return true; } BinlogEventPtr MySQLClient::readOneBinlogEvent() @@ -170,9 +150,4 @@ BinlogEventPtr MySQLClient::readOneBinlogEvent() } } -String MySQLClient::error() -{ - return last_error; -} - } diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 06ec81e6adb..726707dd129 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -28,12 +28,11 @@ class MySQLClient { public: MySQLClient(const String & host_, UInt16 port_, const String & user_, const String & password_); - bool connect(); + void connect(); void disconnect(); - bool ping(); - String error(); + void ping(); - bool startBinlogDump(UInt32 slave_id, String replicate_db, String binlog_file_name, UInt64 binlog_pos); + void startBinlogDump(UInt32 slave_id, String replicate_db, String binlog_file_name, UInt64 binlog_pos); BinlogEventPtr readOneBinlogEvent(); Position getPosition() const { return replication.getPosition(); } @@ -44,7 +43,6 @@ private: String password; bool connected = false; - String last_error; UInt32 client_capability_flags = 0; uint8_t seq = 0; @@ -59,9 +57,9 @@ private: std::optional address; std::shared_ptr packet_sender; - bool handshake(); - bool registerSlaveOnMaster(UInt32 slave_id); - bool writeCommand(char command, String query); + void handshake(); + void registerSlaveOnMaster(UInt32 slave_id); + void writeCommand(char command, String query); }; class WriteCommand : public WritePacket diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index 89cd07d260e..51b067041ef 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -166,18 +166,14 @@ int main(int, char **) { UInt32 slave_id = 9004; MySQLClient slave("127.0.0.1", 9001, "default", "123"); - if (!slave.connect()) - { - std::cerr << "Connect Error: " << slave.error() << std::endl; - return 1; - } - if (!slave.startBinlogDump(slave_id, "", "", 4)) - { - std::cerr << "Connect Error: " << slave.error() << std::endl; - return 1; - } + /// Connect to the master. + slave.connect(); + /// start to dump binlog. + slave.startBinlogDump(slave_id, "", "", 4); + + /// Read one binlog event on by one. while (true) { auto event = slave.readOneBinlogEvent(); From 939bc413ba30a36aff22b1230615e9752d34a335 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 12 May 2020 21:52:30 +0800 Subject: [PATCH 052/374] Move replicate filter to the MySQL server --- src/Core/MySQLClient.cpp | 3 --- src/Core/MySQLReplication.cpp | 17 +++-------------- src/Core/MySQLReplication.h | 5 ----- 3 files changed, 3 insertions(+), 22 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index ecf9e70c7e1..38887ad84b2 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -130,9 +130,6 @@ void MySQLClient::startBinlogDump(UInt32 slave_id, String replicate_db, String b /// This requires MySQL version >=5.6, so results are not checked here. writeCommand(Command::COM_QUERY, "CHANGE REPLICATION FILTER REPLICATE_DO_DB = (" + replicate_db + ")"); - /// Set Filter rule to replication. - replication.setReplicateDatabase(replicate_db); - // Register slave. registerSlaveOnMaster(slave_id); diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 1914c96b3bf..ae7b942d64f 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -710,8 +710,6 @@ namespace MySQLReplication std::cerr << "[DryRun Event]" << std::endl; } - void MySQLFlavor::setReplicateDatabase(String db) { replicate_do_db = std::move(db); } - void MySQLFlavor::readPayloadImpl(ReadBuffer & payload) { UInt16 header = static_cast(*payload.position()); @@ -783,30 +781,21 @@ namespace MySQLReplication } case WRITE_ROWS_EVENT_V1: case WRITE_ROWS_EVENT_V2: { - if (do_replicate()) - event = std::make_shared(table_map); - else - event = std::make_shared(); + event = std::make_shared(table_map); event->parseHeader(payload); event->parseEvent(payload); break; } case DELETE_ROWS_EVENT_V1: case DELETE_ROWS_EVENT_V2: { - if (do_replicate()) - event = std::make_shared(table_map); - else - event = std::make_shared(); + event = std::make_shared(table_map); event->parseHeader(payload); event->parseEvent(payload); break; } case UPDATE_ROWS_EVENT_V1: case UPDATE_ROWS_EVENT_V2: { - if (do_replicate()) - event = std::make_shared(table_map); - else - event = std::make_shared(); + event = std::make_shared(table_map); event->parseHeader(payload); event->parseEvent(payload); break; diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index 54d4a058658..f61670621c4 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -470,7 +470,6 @@ namespace MySQLReplication virtual String getName() const = 0; virtual Position getPosition() const = 0; virtual BinlogEventPtr readOneEvent() = 0; - virtual void setReplicateDatabase(String db) = 0; virtual ~IFlavor() = default; }; @@ -480,17 +479,13 @@ namespace MySQLReplication BinlogEventPtr event; void readPayloadImpl(ReadBuffer & payload) override; - void setReplicateDatabase(String db) override; String getName() const override { return "MySQL"; } Position getPosition() const override { return position; } BinlogEventPtr readOneEvent() override { return event; } private: Position position; - String replicate_do_db; std::shared_ptr table_map; - - inline bool do_replicate() { return (replicate_do_db.empty() || (table_map->schema == replicate_do_db)); } }; } From 8090cf1a00b70b8923e8f4de54e223195d3bd8e7 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 12 May 2020 22:00:37 +0800 Subject: [PATCH 053/374] Add default branch to compressed_decimals read --- src/Core/MySQLClient.cpp | 2 -- src/Core/MySQLReplication.cpp | 2 ++ 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 38887ad84b2..4b420068571 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -75,9 +75,7 @@ void MySQLClient::handshake() packet_sender->receivePacket(packet_response); packet_sender->resetSequenceId(); if (packet_response.getType() == PACKET_ERR) - { throw MySQLClientError(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); - } } void MySQLClient::writeCommand(char command, String query) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index ae7b942d64f..642bc037081 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -573,6 +573,8 @@ namespace MySQLReplication readBigEndianStrict(reader, reinterpret_cast(&val), 4); break; } + default: + break; } format += std::to_string(val); } From 644667adff10bbf7e3eb51d76b303928c843a8c9 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 12 May 2020 22:12:43 +0800 Subject: [PATCH 054/374] Change the method print() to dump() --- src/Core/MySQLReplication.cpp | 38 +++++++++++++++---------------- src/Core/MySQLReplication.h | 18 +++++++-------- src/Core/tests/mysql_protocol.cpp | 8 +++---- 3 files changed, 32 insertions(+), 32 deletions(-) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 642bc037081..db749fec170 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -26,7 +26,7 @@ namespace MySQLReplication payload.readStrict(reinterpret_cast(&flags), 2); } - void EventHeader::print() const + void EventHeader::dump() const { std::cerr << "\n=== " << to_string(this->type) << " ===" << std::endl; std::cerr << "Timestamp: " << this->timestamp << std::endl; @@ -52,9 +52,9 @@ namespace MySQLReplication payload.readStrict(reinterpret_cast(event_type_header_length.data()), len); } - void FormatDescriptionEvent::print() const + void FormatDescriptionEvent::dump() const { - header.print(); + header.dump(); std::cerr << "Binlog Version: " << this->binlog_version << std::endl; std::cerr << "Server Version: " << this->server_version << std::endl; std::cerr << "Create Timestamp: " << this->create_timestamp << std::endl; @@ -70,9 +70,9 @@ namespace MySQLReplication payload.readStrict(reinterpret_cast(next_binlog.data()), len); } - void RotateEvent::print() const + void RotateEvent::dump() const { - header.print(); + header.dump(); std::cerr << "Position: " << this->position << std::endl; std::cerr << "Next Binlog: " << this->next_binlog << std::endl; } @@ -108,9 +108,9 @@ namespace MySQLReplication } } - void QueryEvent::print() const + void QueryEvent::dump() const { - header.print(); + header.dump(); std::cerr << "Thread ID: " << this->thread_id << std::endl; std::cerr << "Execution Time: " << this->exec_time << std::endl; std::cerr << "Schema Len: " << this->schema_len << std::endl; @@ -123,9 +123,9 @@ namespace MySQLReplication void XIDEvent::parseImpl(ReadBuffer & payload) { payload.readStrict(reinterpret_cast(&xid), 8); } - void XIDEvent::print() const + void XIDEvent::dump() const { - header.print(); + header.dump(); std::cerr << "XID: " << this->xid << std::endl; } @@ -224,9 +224,9 @@ namespace MySQLReplication } } - void TableMapEvent::print() const + void TableMapEvent::dump() const { - header.print(); + header.dump(); std::cerr << "Table ID: " << this->table_id << std::endl; std::cerr << "Flags: " << this->flags << std::endl; std::cerr << "Schema Len: " << this->schema_len << std::endl; @@ -530,19 +530,19 @@ namespace MySQLReplication if (compressed_integers != 0) { Int64 val = 0; - UInt32 to_read = compressed_byte_map[compressed_integers]; + UInt8 to_read = compressed_byte_map[compressed_integers]; readBigEndianStrict(reader, reinterpret_cast(&val), to_read); format += std::to_string(val); } - for (auto k = 0; k < uncompressed_integers; k++) + for (auto k = 0U; k < uncompressed_integers; k++) { UInt32 val = 0; readBigEndianStrict(reader, reinterpret_cast(&val), 4); format += std::to_string(val); } format += "."; - for (auto k = 0; k < uncompressed_decimals; k++) + for (auto k = 0U; k < uncompressed_decimals; k++) { UInt32 val = 0; reader.readStrict(reinterpret_cast(&val), 4); @@ -554,7 +554,7 @@ namespace MySQLReplication { Int64 val = 0; String compressed_buff; - UInt32 to_read = compressed_byte_map[compressed_decimals]; + UInt8 to_read = compressed_byte_map[compressed_decimals]; switch (to_read) { case 1: { @@ -686,11 +686,11 @@ namespace MySQLReplication rows.push_back(row); } - void RowsEvent::print() const + void RowsEvent::dump() const { FieldVisitorToString to_string; - header.print(); + header.dump(); std::cerr << "Schema: " << this->schema << std::endl; std::cerr << "Table: " << this->table << std::endl; for (auto i = 0U; i < rows.size(); i++) @@ -706,9 +706,9 @@ namespace MySQLReplication } } - void DryRunEvent::print() const + void DryRunEvent::dump() const { - header.print(); + header.dump(); std::cerr << "[DryRun Event]" << std::endl; } diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index f61670621c4..544f44bbcaf 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -289,7 +289,7 @@ namespace MySQLReplication UInt32 log_pos; UInt16 flags; - void print() const; + void dump() const; void parse(ReadBuffer & payload); }; @@ -299,7 +299,7 @@ namespace MySQLReplication EventHeader header; virtual ~EventBase() = default; - virtual void print() const = 0; + virtual void dump() const = 0; virtual void parseHeader(ReadBuffer & payload) { header.parse(payload); } virtual void parseEvent(ReadBuffer & payload) { parseImpl(payload); } virtual MySQLEventType type() const { return MYSQL_UNHANDLED_EVENT; } @@ -317,7 +317,7 @@ namespace MySQLReplication UInt8 event_header_length; String event_type_header_length; - void print() const override; + void dump() const override; void parseImpl(ReadBuffer & payload) override; private: @@ -330,7 +330,7 @@ namespace MySQLReplication UInt64 position; String next_binlog; - void print() const override; + void dump() const override; protected: void parseImpl(ReadBuffer & payload) override; @@ -356,7 +356,7 @@ namespace MySQLReplication String query; QueryType typ = DDL; - void print() const override; + void dump() const override; MySQLEventType type() const override { return MYSQL_QUERY_EVENT; } protected: @@ -368,7 +368,7 @@ namespace MySQLReplication protected: UInt64 xid; - void print() const override; + void dump() const override; void parseImpl(ReadBuffer & payload) override; }; @@ -386,7 +386,7 @@ namespace MySQLReplication std::vector column_meta; String null_bitmap; - void print() const override; + void dump() const override; protected: void parseImpl(ReadBuffer & payload) override; @@ -408,7 +408,7 @@ namespace MySQLReplication table = table_map->table; } - void print() const override; + void dump() const override; protected: UInt64 table_id; @@ -447,7 +447,7 @@ namespace MySQLReplication class DryRunEvent : public EventBase { - void print() const override; + void dump() const override; protected: void parseImpl(ReadBuffer & payload) override; diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index 51b067041ef..96de2c5ce1d 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -181,7 +181,7 @@ int main(int, char **) { case MYSQL_QUERY_EVENT: { auto binlogEvent = std::static_pointer_cast(event); - binlogEvent->print(); + binlogEvent->dump(); Position pos = slave.getPosition(); std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; @@ -189,7 +189,7 @@ int main(int, char **) } case MYSQL_WRITE_ROWS_EVENT: { auto binlogEvent = std::static_pointer_cast(event); - binlogEvent->print(); + binlogEvent->dump(); Position pos = slave.getPosition(); std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; @@ -197,7 +197,7 @@ int main(int, char **) } case MYSQL_UPDATE_ROWS_EVENT: { auto binlogEvent = std::static_pointer_cast(event); - binlogEvent->print(); + binlogEvent->dump(); Position pos = slave.getPosition(); std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; @@ -205,7 +205,7 @@ int main(int, char **) } case MYSQL_DELETE_ROWS_EVENT: { auto binlogEvent = std::static_pointer_cast(event); - binlogEvent->print(); + binlogEvent->dump(); Position pos = slave.getPosition(); std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; From ec9345785bb4d634acfbe8b531f061c6820ad898 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Wed, 13 May 2020 12:18:10 +0800 Subject: [PATCH 055/374] Change flavor event to private --- src/Core/MySQLReplication.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index 544f44bbcaf..caef184add2 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -476,8 +476,6 @@ namespace MySQLReplication class MySQLFlavor : public IFlavor { public: - BinlogEventPtr event; - void readPayloadImpl(ReadBuffer & payload) override; String getName() const override { return "MySQL"; } Position getPosition() const override { return position; } @@ -485,6 +483,7 @@ namespace MySQLReplication private: Position position; + BinlogEventPtr event; std::shared_ptr table_map; }; } From d2aa1f9a473a1bb8cf4b78c9ebf79ee2f393426c Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Thu, 14 May 2020 16:21:44 +0800 Subject: [PATCH 056/374] Disable XA ROLLBACK and SAVEPOINT type for QueryEvent --- src/Core/MySQLReplication.cpp | 15 ++++++++++----- src/Core/MySQLReplication.h | 2 +- src/Core/tests/mysql_protocol.cpp | 1 + 3 files changed, 12 insertions(+), 6 deletions(-) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index db749fec170..a2e081823b3 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -97,14 +97,19 @@ namespace MySQLReplication = header.event_size - EVENT_HEADER_LENGTH - 4 - 4 - 1 - 2 - 2 - status_len - schema_len - 1 - CHECKSUM_CRC32_SIGNATURE_LENGTH; query.resize(len); payload.readStrict(reinterpret_cast(query.data()), len); - - if (query == "BEGIN") + if (query.rfind("BEGIN", 0) == 0) { typ = BEGIN; } - else if (query == "SAVEPOINT") + else if (query.rfind("XA", 0) == 0) { - typ = SAVEPOINT; + if (query.rfind("XA ROLLBACK", 0) == 0) + throw ReplicationError("ParseQueryEvent: Unsupported query event:" + query, ErrorCodes::UNKNOWN_EXCEPTION); + typ = XA; + } + else if (query.rfind("SAVEPOINT", 0) == 0) + { + throw ReplicationError("ParseQueryEvent: Unsupported query event:" + query, ErrorCodes::UNKNOWN_EXCEPTION); } } @@ -756,7 +761,7 @@ namespace MySQLReplication switch (query->typ) { case BEGIN: - case SAVEPOINT: { + case XA: { event = std::make_shared(); break; } diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index caef184add2..df0eb36e395 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -340,7 +340,7 @@ namespace MySQLReplication { DDL = 0, BEGIN = 1, - SAVEPOINT = 2 + XA = 2 }; class QueryEvent : public EventBase diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index 96de2c5ce1d..931884a1cee 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -212,6 +212,7 @@ int main(int, char **) break; } default: + event->dump(); break; } } From 882a773264e0f911d9937a36b3700e94749d2dd2 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Fri, 15 May 2020 14:54:26 +0800 Subject: [PATCH 057/374] Support database replicate filter --- src/Core/MySQLClient.cpp | 7 +++-- src/Core/MySQLReplication.cpp | 43 ++++++++++++++++++++++--------- src/Core/MySQLReplication.h | 5 ++++ src/Core/tests/mysql_protocol.cpp | 3 +-- 4 files changed, 40 insertions(+), 18 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 4b420068571..3400229855a 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -124,13 +124,12 @@ void MySQLClient::startBinlogDump(UInt32 slave_id, String replicate_db, String b UInt64 period_ns = (30 * 1e9); writeCommand(Command::COM_QUERY, "SET @master_heartbeat_period = " + std::to_string(period_ns)); - /// Set replication filter to master - /// This requires MySQL version >=5.6, so results are not checked here. - writeCommand(Command::COM_QUERY, "CHANGE REPLICATION FILTER REPLICATE_DO_DB = (" + replicate_db + ")"); - // Register slave. registerSlaveOnMaster(slave_id); + /// Set Filter rule to replication. + replication.setReplicateDatabase(replicate_db); + binlog_pos = binlog_pos < 4 ? 4 : binlog_pos; BinlogDump binlog_dump(binlog_pos, binlog_file_name, slave_id); packet_sender->sendPacket(binlog_dump, true); diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index a2e081823b3..decc31f3140 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -758,15 +758,23 @@ namespace MySQLReplication event->parseEvent(payload); auto query = std::static_pointer_cast(event); - switch (query->typ) + if (query->schema == replicate_do_db) { - case BEGIN: - case XA: { - event = std::make_shared(); - break; + switch (query->typ) + { + case BEGIN: + case XA: { + event = std::make_shared(); + break; + } + default: + position.updateLogPos(event->header.log_pos); } - default: - position.updateLogPos(event->header.log_pos); + } + else + { + event = std::make_shared(); + position.updateLogPos(event->header.log_pos); } break; } @@ -781,28 +789,39 @@ namespace MySQLReplication event = std::make_shared(); event->parseHeader(payload); event->parseEvent(payload); - - table_map = std::static_pointer_cast(event); position.updateLogPos(event->header.log_pos); + table_map = std::static_pointer_cast(event); break; } case WRITE_ROWS_EVENT_V1: case WRITE_ROWS_EVENT_V2: { - event = std::make_shared(table_map); + if (do_replicate()) + event = std::make_shared(table_map); + else + event = std::make_shared(); + event->parseHeader(payload); event->parseEvent(payload); break; } case DELETE_ROWS_EVENT_V1: case DELETE_ROWS_EVENT_V2: { - event = std::make_shared(table_map); + if (do_replicate()) + event = std::make_shared(table_map); + else + event = std::make_shared(); + event->parseHeader(payload); event->parseEvent(payload); break; } case UPDATE_ROWS_EVENT_V1: case UPDATE_ROWS_EVENT_V2: { - event = std::make_shared(table_map); + if (do_replicate()) + event = std::make_shared(table_map); + else + event = std::make_shared(); + event->parseHeader(payload); event->parseEvent(payload); break; diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index df0eb36e395..4537afb51a3 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -470,6 +470,7 @@ namespace MySQLReplication virtual String getName() const = 0; virtual Position getPosition() const = 0; virtual BinlogEventPtr readOneEvent() = 0; + virtual void setReplicateDatabase(String db) = 0; virtual ~IFlavor() = default; }; @@ -480,11 +481,15 @@ namespace MySQLReplication String getName() const override { return "MySQL"; } Position getPosition() const override { return position; } BinlogEventPtr readOneEvent() override { return event; } + void setReplicateDatabase(String db) override { replicate_do_db = std::move(db); } private: Position position; BinlogEventPtr event; + String replicate_do_db; std::shared_ptr table_map; + + inline bool do_replicate() { return (replicate_do_db.empty() || table_map->schema == replicate_do_db); } }; } diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index 931884a1cee..9564bdb544c 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -171,7 +171,7 @@ int main(int, char **) slave.connect(); /// start to dump binlog. - slave.startBinlogDump(slave_id, "", "", 4); + slave.startBinlogDump(slave_id, "dbtest", "", 4); /// Read one binlog event on by one. while (true) @@ -212,7 +212,6 @@ int main(int, char **) break; } default: - event->dump(); break; } } From 3477a467e685265a2c474e5229157d5fa06baa5a Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Sat, 23 May 2020 14:38:30 +0800 Subject: [PATCH 058/374] Add master connect flags and change DryRunEvent parse to ignore --- src/Core/MySQLReplication.cpp | 7 +------ src/Core/tests/mysql_protocol.cpp | 30 +++++++++++++++++++++++++++--- 2 files changed, 28 insertions(+), 9 deletions(-) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index decc31f3140..0c74f9a4f34 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -704,12 +704,7 @@ namespace MySQLReplication } } - void DryRunEvent::parseImpl(ReadBuffer & payload) - { - while (payload.next()) - { - } - } + void DryRunEvent::parseImpl(ReadBuffer & payload) { payload.ignore(header.event_size - EVENT_HEADER_LENGTH); } void DryRunEvent::dump() const { diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index 9564bdb544c..a57773e4eb9 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -5,8 +5,9 @@ #include #include +#include -int main(int, char **) +int main(int argc, char ** argv) { using namespace DB; using namespace MySQLProtocol; @@ -164,14 +165,37 @@ int main(int, char **) { try { + boost::program_options::options_description desc("Allowed options"); + desc.add_options()("host", boost::program_options::value()->required(), "master host")( + "port", boost::program_options::value()->required(), "master port")( + "user", boost::program_options::value()->required(), "master user")( + "password", boost::program_options::value()->required(), "master password")( + "db", boost::program_options::value()->required(), "replicate do db"); + + boost::program_options::variables_map options; + boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); + if (argc == 0) + { + return 1; + } + + auto host = options.at("host").as(); + auto port = options.at("port").as(); + auto master_user = options.at("user").as(); + auto master_password = options.at("password").as(); + auto replicate_db = options.at("db").as(); + + std::cerr << "Master Host: " << host << ", Port: " << port << ", User: " << master_user << ", Password: " << master_password + << ", Replicate DB: " << replicate_db << std::endl; + UInt32 slave_id = 9004; - MySQLClient slave("127.0.0.1", 9001, "default", "123"); + MySQLClient slave(host, port, master_user, master_password); /// Connect to the master. slave.connect(); /// start to dump binlog. - slave.startBinlogDump(slave_id, "dbtest", "", 4); + slave.startBinlogDump(slave_id, replicate_db, "", 4); /// Read one binlog event on by one. while (true) From 2a0e79c145bb1d8da9256928aa1ef9aa3a4ba7c2 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Sat, 23 May 2020 18:42:21 +0800 Subject: [PATCH 059/374] Fix MySQL 8.0 optional metadata fields read in TableMapEvent --- src/Core/MySQLReplication.cpp | 8 ++++++-- src/Core/MySQLReplication.h | 2 +- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 0c74f9a4f34..bdaf96b4ddd 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -161,8 +161,12 @@ namespace MySQLReplication readLengthEncodedString(meta, payload); parseMeta(meta); - size_t len = (column_count + 8) / 7; - payload.readStrict(reinterpret_cast(null_bitmap.data()), len); + size_t null_bitmap_size = (column_count + 8) / 7; + readBitmap(payload, null_bitmap, null_bitmap_size); + + /// Ignore MySQL 8.0 optional metadata fields. + /// https://mysqlhighavailability.com/more-metadata-is-written-into-binary-log/ + payload.ignore(payload.available() - CHECKSUM_CRC32_SIGNATURE_LENGTH); } void TableMapEvent::parseMeta(String meta) diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index 4537afb51a3..420bd8fdc02 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -384,7 +384,7 @@ namespace MySQLReplication UInt32 column_count; std::vector column_type; std::vector column_meta; - String null_bitmap; + Bitmap null_bitmap; void dump() const override; From 5f80db756910b00fa8ff17dc411d6b63176ed798 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Sat, 23 May 2020 23:31:58 +0800 Subject: [PATCH 060/374] Fix MySQL 8.0 AuthSwitch packet read --- src/Core/MySQLClient.cpp | 5 ++++- src/Core/MySQLProtocol.h | 37 +++++++++++++++++++++++++++++++++-- src/Core/MySQLReplication.cpp | 2 +- 3 files changed, 40 insertions(+), 4 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 3400229855a..4f3186fd2f7 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -71,11 +71,14 @@ void MySQLClient::handshake() client_capability_flags, max_packet_size, charset_utf8, user, "", auth_plugin_data, mysql_native_password); packet_sender->sendPacket(handshake_response, true); - PacketResponse packet_response(client_capability_flags); + PacketResponse packet_response(client_capability_flags, true); packet_sender->receivePacket(packet_response); packet_sender->resetSequenceId(); + if (packet_response.getType() == PACKET_ERR) throw MySQLClientError(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); + else if (packet_response.getType() == PACKET_AUTH_SWITCH) + throw MySQLClientError("Access denied for user " + user, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); } void MySQLClient::writeCommand(char command, String query) diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index ae97eb3dfe1..1615ae7c7c6 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -148,6 +148,7 @@ enum ResponsePacketType PACKET_OK = 0x00, PACKET_ERR = 0xff, PACKET_EOF = 0xfe, + PACKET_AUTH_SWITCH = 0xfe, PACKET_LOCALINFILE = 0xfb, }; @@ -903,6 +904,24 @@ protected: } }; +class AuthSwitch_Packet : public ReadPacket +{ +public: + String plugin_name; + + AuthSwitch_Packet() { } + + void readPayloadImpl(ReadBuffer & payload) override + { + payload.readStrict(reinterpret_cast(&header), 1); + assert(header == 0xfe); + readStringUntilEOF(plugin_name, payload); + } + +private: + UInt8 header = 0x00; +}; + class ERR_Packet : public WritePacket, public ReadPacket { public: @@ -959,9 +978,14 @@ public: OK_Packet ok; ERR_Packet err; EOF_Packet eof; + AuthSwitch_Packet auth_switch; UInt64 column_length = 0; PacketResponse(UInt32 server_capability_flags_) : ok(OK_Packet(server_capability_flags_)) { } + PacketResponse(UInt32 server_capability_flags_, bool is_handshake_) + : ok(OK_Packet(server_capability_flags_)), is_handshake(is_handshake_) + { + } void readPayloadImpl(ReadBuffer & payload) override { @@ -977,8 +1001,16 @@ public: err.readPayloadImpl(payload); break; case PACKET_EOF: - packetType = PACKET_EOF; - eof.readPayloadImpl(payload); + if (is_handshake) + { + packetType = PACKET_AUTH_SWITCH; + auth_switch.readPayloadImpl(payload); + } + else + { + packetType = PACKET_EOF; + eof.readPayloadImpl(payload); + } break; case PACKET_LOCALINFILE: packetType = PACKET_LOCALINFILE; @@ -992,6 +1024,7 @@ public: ResponsePacketType getType() { return packetType; } private: + bool is_handshake = false; ResponsePacketType packetType = PACKET_OK; }; diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index bdaf96b4ddd..e73a920ac88 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -161,7 +161,7 @@ namespace MySQLReplication readLengthEncodedString(meta, payload); parseMeta(meta); - size_t null_bitmap_size = (column_count + 8) / 7; + size_t null_bitmap_size = (column_count + 7) / 8; readBitmap(payload, null_bitmap, null_bitmap_size); /// Ignore MySQL 8.0 optional metadata fields. From c53a48fe010390d8ddbd9f0bd1bfc5e21d614025 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 29 Apr 2020 02:36:43 +0800 Subject: [PATCH 061/374] ISSUES-4006 support parse MySQL create query --- src/CMakeLists.txt | 1 + src/Databases/DatabaseFactory.cpp | 47 +--- .../MySQL/DatabaseMaterializeMySQL.cpp | 97 ++++++++ .../MySQL/DatabaseMaterializeMySQL.h | 30 +++ src/Databases/{ => MySQL}/DatabaseMySQL.cpp | 3 +- src/Databases/{ => MySQL}/DatabaseMySQL.h | 0 src/Databases/MySQL/MasterStatusInfo.cpp | 6 + src/Databases/MySQL/MasterStatusInfo.h | 27 ++ src/Databases/MySQL/createMySQLDatabase.cpp | 84 +++++++ src/Databases/MySQL/createMySQLDatabase.h | 19 ++ src/Parsers/CMakeLists.txt | 1 + src/Parsers/MySQL/ASTCreateDefines.cpp | 95 +++++++ src/Parsers/MySQL/ASTCreateDefines.h | 37 +++ src/Parsers/MySQL/ASTCreateQuery.cpp | 128 ++++++++++ src/Parsers/MySQL/ASTCreateQuery.h | 40 +++ src/Parsers/MySQL/ASTDeclareColumn.cpp | 98 ++++++++ src/Parsers/MySQL/ASTDeclareColumn.h | 37 +++ src/Parsers/MySQL/ASTDeclareConstraint.cpp | 74 ++++++ src/Parsers/MySQL/ASTDeclareConstraint.h | 35 +++ src/Parsers/MySQL/ASTDeclareIndex.cpp | 235 ++++++++++++++++++ src/Parsers/MySQL/ASTDeclareIndex.h | 43 ++++ src/Parsers/MySQL/ASTDeclareOption.cpp | 142 +++++++++++ src/Parsers/MySQL/ASTDeclareOption.h | 76 ++++++ src/Parsers/MySQL/ASTDeclarePartition.cpp | 127 ++++++++++ src/Parsers/MySQL/ASTDeclarePartition.h | 35 +++ .../MySQL/ASTDeclarePartitionOptions.cpp | 184 ++++++++++++++ .../MySQL/ASTDeclarePartitionOptions.h | 40 +++ src/Parsers/MySQL/ASTDeclareReference.cpp | 105 ++++++++ src/Parsers/MySQL/ASTDeclareReference.h | 52 ++++ src/Parsers/MySQL/ASTDeclareSubPartition.cpp | 66 +++++ src/Parsers/MySQL/ASTDeclareSubPartition.h | 33 +++ src/Parsers/MySQL/ASTDeclareTableOptions.cpp | 102 ++++++++ src/Parsers/MySQL/ASTDeclareTableOptions.h | 21 ++ .../MySQL/tests/gtest_column_parser.cpp | 60 +++++ .../MySQL/tests/gtest_constraint_parser.cpp | 57 +++++ .../MySQL/tests/gtest_create_parser.cpp | 33 +++ .../MySQL/tests/gtest_index_parser.cpp | 125 ++++++++++ .../tests/gtest_partition_options_parser.cpp | 167 +++++++++++++ .../MySQL/tests/gtest_partition_parser.cpp | 149 +++++++++++ .../MySQL/tests/gtest_reference_parser.cpp | 92 +++++++ .../MySQL/tests/gtest_subpartition_parser.cpp | 45 ++++ .../tests/gtest_table_options_parser.cpp | 64 +++++ 42 files changed, 2868 insertions(+), 44 deletions(-) create mode 100644 src/Databases/MySQL/DatabaseMaterializeMySQL.cpp create mode 100644 src/Databases/MySQL/DatabaseMaterializeMySQL.h rename src/Databases/{ => MySQL}/DatabaseMySQL.cpp (99%) rename src/Databases/{ => MySQL}/DatabaseMySQL.h (100%) create mode 100644 src/Databases/MySQL/MasterStatusInfo.cpp create mode 100644 src/Databases/MySQL/MasterStatusInfo.h create mode 100644 src/Databases/MySQL/createMySQLDatabase.cpp create mode 100644 src/Databases/MySQL/createMySQLDatabase.h create mode 100644 src/Parsers/MySQL/ASTCreateDefines.cpp create mode 100644 src/Parsers/MySQL/ASTCreateDefines.h create mode 100644 src/Parsers/MySQL/ASTCreateQuery.cpp create mode 100644 src/Parsers/MySQL/ASTCreateQuery.h create mode 100644 src/Parsers/MySQL/ASTDeclareColumn.cpp create mode 100644 src/Parsers/MySQL/ASTDeclareColumn.h create mode 100644 src/Parsers/MySQL/ASTDeclareConstraint.cpp create mode 100644 src/Parsers/MySQL/ASTDeclareConstraint.h create mode 100644 src/Parsers/MySQL/ASTDeclareIndex.cpp create mode 100644 src/Parsers/MySQL/ASTDeclareIndex.h create mode 100644 src/Parsers/MySQL/ASTDeclareOption.cpp create mode 100644 src/Parsers/MySQL/ASTDeclareOption.h create mode 100644 src/Parsers/MySQL/ASTDeclarePartition.cpp create mode 100644 src/Parsers/MySQL/ASTDeclarePartition.h create mode 100644 src/Parsers/MySQL/ASTDeclarePartitionOptions.cpp create mode 100644 src/Parsers/MySQL/ASTDeclarePartitionOptions.h create mode 100644 src/Parsers/MySQL/ASTDeclareReference.cpp create mode 100644 src/Parsers/MySQL/ASTDeclareReference.h create mode 100644 src/Parsers/MySQL/ASTDeclareSubPartition.cpp create mode 100644 src/Parsers/MySQL/ASTDeclareSubPartition.h create mode 100644 src/Parsers/MySQL/ASTDeclareTableOptions.cpp create mode 100644 src/Parsers/MySQL/ASTDeclareTableOptions.h create mode 100644 src/Parsers/MySQL/tests/gtest_column_parser.cpp create mode 100644 src/Parsers/MySQL/tests/gtest_constraint_parser.cpp create mode 100644 src/Parsers/MySQL/tests/gtest_create_parser.cpp create mode 100644 src/Parsers/MySQL/tests/gtest_index_parser.cpp create mode 100644 src/Parsers/MySQL/tests/gtest_partition_options_parser.cpp create mode 100644 src/Parsers/MySQL/tests/gtest_partition_parser.cpp create mode 100644 src/Parsers/MySQL/tests/gtest_reference_parser.cpp create mode 100644 src/Parsers/MySQL/tests/gtest_subpartition_parser.cpp create mode 100644 src/Parsers/MySQL/tests/gtest_table_options_parser.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 826b5ad54de..74f0a03408e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -144,6 +144,7 @@ add_object_library(clickhouse_compression Compression) add_object_library(clickhouse_datastreams DataStreams) add_object_library(clickhouse_datatypes DataTypes) add_object_library(clickhouse_databases Databases) +add_object_library(clickhouse_databases_mysql Databases/MySQL) add_object_library(clickhouse_disks Disks) add_object_library(clickhouse_interpreters Interpreters) add_object_library(clickhouse_interpreters_clusterproxy Interpreters/ClusterProxy) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 3b84dfa8949..1887317c05d 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -1,16 +1,17 @@ +#include + #include #include -#include #include #include #include +#include #include #include #include #include #include #include -#include "DatabaseFactory.h" #include #include @@ -18,12 +19,6 @@ # include "config_core.h" #endif -#if USE_MYSQL -# include -# include -#endif - - namespace DB { @@ -94,42 +89,8 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String return std::make_shared(database_name, context); #if USE_MYSQL - else if (engine_name == "MySQL") - { - const ASTFunction * engine = engine_define->engine; - - if (!engine->arguments || engine->arguments->children.size() != 4) - throw Exception("MySQL Database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.", - ErrorCodes::BAD_ARGUMENTS); - - - ASTs & arguments = engine->arguments->children; - arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], context); - - const auto & host_name_and_port = safeGetLiteralValue(arguments[0], "MySQL"); - const auto & database_name_in_mysql = safeGetLiteralValue(arguments[1], "MySQL"); - const auto & mysql_user_name = safeGetLiteralValue(arguments[2], "MySQL"); - const auto & mysql_user_password = safeGetLiteralValue(arguments[3], "MySQL"); - - try - { - const auto & [remote_host_name, remote_port] = parseAddress(host_name_and_port, 3306); - auto mysql_pool = mysqlxx::Pool(database_name_in_mysql, remote_host_name, mysql_user_name, mysql_user_password, remote_port); - - auto mysql_database = std::make_shared( - context, database_name, metadata_path, engine_define, database_name_in_mysql, std::move(mysql_pool)); - - mysql_database->empty(); /// test database is works fine. - return mysql_database; - } - catch (...) - { - const auto & exception_message = getCurrentExceptionMessage(true); - throw Exception("Cannot create MySQL database, because " + exception_message, ErrorCodes::CANNOT_CREATE_DATABASE); - } - } - + return createMySQLDatabase(database_name, metadata_path, engine_define, context); #endif else if (engine_name == "Lazy") diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp new file mode 100644 index 00000000000..65e5a76ae98 --- /dev/null +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -0,0 +1,97 @@ +//#include +// +//#include +//#include +//#include +//#include +//#include +//#include +//#include +// +//namespace DB +//{ +// +//static MasterStatusInfo fetchMasterStatus(const mysqlxx::PoolWithFailover::Entry & connection) +//{ +// Block header +// { +// {std::make_shared(), "File"}, +// {std::make_shared(), "Position"}, +// {std::make_shared(), "Binlog_Do_DB"}, +// {std::make_shared(), "Binlog_Ignore_DB"}, +// {std::make_shared(), "Executed_Gtid_Set"}, +// }; +// +// MySQLBlockInputStream input(connection, "SHOW MASTER STATUS;", header, DEFAULT_BLOCK_SIZE); +// Block master_status = input.read(); +// +// if (!master_status || master_status.rows() != 1) +// throw Exception("Unable to get master status from MySQL.", ErrorCodes::LOGICAL_ERROR); +// +// return MasterStatusInfo +// { +// (*master_status.getByPosition(0).column)[0].safeGet(), +// (*master_status.getByPosition(1).column)[0].safeGet(), +// (*master_status.getByPosition(2).column)[0].safeGet(), +// (*master_status.getByPosition(3).column)[0].safeGet(), +// (*master_status.getByPosition(4).column)[0].safeGet() +// }; +//} +// +//static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entry & connection, const std::string & database) +//{ +// Block header{{std::make_shared(), "table_name"}}; +// String query = "SELECT TABLE_NAME AS table_name FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA = " + quoteString(database); +// +// std::vector tables_in_db; +// MySQLBlockInputStream input(connection, query, header, DEFAULT_BLOCK_SIZE); +// +// while (Block block = input.read()) +// { +// tables_in_db.reserve(tables_in_db.size() + block.rows()); +// for (size_t index = 0; index < block.rows(); ++index) +// tables_in_db.emplace_back((*block.getByPosition(0).column)[index].safeGet()); +// } +// +// return tables_in_db; +//} +// +//DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( +// const Context & context, const String & database_name_, const String & metadata_path_, +// const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_) +// : IDatabase(database_name_) +// , global_context(context.getGlobalContext()), metadata_path(metadata_path_) +// , database_engine_define(database_engine_define_->clone()), mysql_database_name(mysql_database_name_), pool(std::move(pool_)) +//{ +// try +// { +// mysqlxx::PoolWithFailover::Entry connection = pool.get(); +// +// connection->query("FLUSH TABLES;").execute(); +// connection->query("FLUSH TABLES WITH READ LOCK;").execute(); +// +// MasterStatusInfo master_status = fetchMasterStatus(connection); +// connection->query("SET SESSION TRANSACTION ISOLATION LEVEL REPEATABLE READ;").execute(); +// connection->query("START TRANSACTION /*!40100 WITH CONSISTENT SNAPSHOT */;").execute(); +// +// std::vector tables_in_db = fetchTablesInDB(connection, mysql_database_name); +// connection->query("UNLOCK TABLES;"); +// +// for (const auto & dumping_table_name : tables_in_db) +// { +// /// TODO: 查询表结构, 根据不同的模式创建对应的表(暂时只支持多version即可) +//// connection->query("SHOW CREATE TABLE " + doubleQuoteString()) +// MySQLBlockInputStream input( +// "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(dumping_table_name)); +// /// TODO: 查询所有数据写入对应表中(全量dump) +// /// TODO: 启动slave, 监听事件 +// } +// } +// catch (...) +// { +// throw; +// } +//} +// +// +//} diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h new file mode 100644 index 00000000000..bb7e6193451 --- /dev/null +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -0,0 +1,30 @@ +//#pragma once +// +//#include +//#include +//#include +//#include +//#include +// +//namespace DB +//{ +// +//class DatabaseMaterializeMySQL : public IDatabase +//{ +//public: +// DatabaseMaterializeMySQL( +// const Context & context, const String & database_name_, const String & metadata_path_, +// const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_); +// +// String getEngineName() const override { return "MySQL"; } +// +//private: +// const Context & global_context; +// String metadata_path; +// ASTPtr database_engine_define; +// String mysql_database_name; +// +// mutable mysqlxx::Pool pool; +//}; +// +//} diff --git a/src/Databases/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp similarity index 99% rename from src/Databases/DatabaseMySQL.cpp rename to src/Databases/MySQL/DatabaseMySQL.cpp index 3b026bf9468..ff12446331a 100644 --- a/src/Databases/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -9,7 +9,7 @@ # include # include # include -# include +# include # include # include # include @@ -70,6 +70,7 @@ DatabaseMySQL::DatabaseMySQL( , database_name_in_mysql(database_name_in_mysql_) , mysql_pool(std::move(pool)) { + empty(); /// test database is works fine. } bool DatabaseMySQL::empty() const diff --git a/src/Databases/DatabaseMySQL.h b/src/Databases/MySQL/DatabaseMySQL.h similarity index 100% rename from src/Databases/DatabaseMySQL.h rename to src/Databases/MySQL/DatabaseMySQL.h diff --git a/src/Databases/MySQL/MasterStatusInfo.cpp b/src/Databases/MySQL/MasterStatusInfo.cpp new file mode 100644 index 00000000000..5c53db22bc4 --- /dev/null +++ b/src/Databases/MySQL/MasterStatusInfo.cpp @@ -0,0 +1,6 @@ +#include + +namespace DB +{ + +} diff --git a/src/Databases/MySQL/MasterStatusInfo.h b/src/Databases/MySQL/MasterStatusInfo.h new file mode 100644 index 00000000000..aea50aa3bdd --- /dev/null +++ b/src/Databases/MySQL/MasterStatusInfo.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include + +namespace DB +{ + +struct MasterStatusInfo +{ + String binlog_file; + UInt64 binlog_position; + String binlog_do_db; + String binlog_ignore_db; + String executed_gtid_set; + + MasterStatusInfo( + String binlog_file_, UInt64 binlog_position_, String binlog_do_db_, String binlog_ignore_db_, String executed_gtid_set_); + + +}; + + +std::shared_ptr fetchMasterStatusInfo(mysqlxx::Connection * connection); + +} + diff --git a/src/Databases/MySQL/createMySQLDatabase.cpp b/src/Databases/MySQL/createMySQLDatabase.cpp new file mode 100644 index 00000000000..24417d83791 --- /dev/null +++ b/src/Databases/MySQL/createMySQLDatabase.cpp @@ -0,0 +1,84 @@ +#include + +#if USE_MYSQL + +#include +#include +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int CANNOT_CREATE_DATABASE; +} + +static inline String safeGetLiteralValue(const ASTPtr & ast) +{ + if (!ast || !ast->as()) + throw Exception("Database engine MySQL requested literal argument.", ErrorCodes::BAD_ARGUMENTS); + + return ast->as()->value.safeGet(); +} + +/*static inline bool materializeMySQLDatabase(const ASTSetQuery * settings) +{ + if (!settings || settings->changes.empty()) + return false; + + for (const auto & change : settings->changes) + { + if (change.name == "materialize_data") + { + if (change.value.getType() == Field::Types::String) + return change.value.safeGet() == "true"; ///TODO: ignore case + } + + } + return false; +}*/ + +DatabasePtr createMySQLDatabase(const String & database_name, const String & metadata_path, const ASTStorage * define, Context & context) +{ + const ASTFunction * engine = define->engine; + if (!engine->arguments || engine->arguments->children.size() != 4) + throw Exception( "MySQL Database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.", + ErrorCodes::BAD_ARGUMENTS); + + ASTs & arguments = engine->arguments->children; + arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], context); + + const auto & host_name_and_port = safeGetLiteralValue(arguments[0]); + const auto & mysql_database_name = safeGetLiteralValue(arguments[1]); + const auto & mysql_user_name = safeGetLiteralValue(arguments[2]); + const auto & mysql_user_password = safeGetLiteralValue(arguments[3]); + + try + { + const auto & [remote_host_name, remote_port] = parseAddress(host_name_and_port, 3306); + auto mysql_pool = mysqlxx::Pool(mysql_database_name, remote_host_name, mysql_user_name, mysql_user_password, remote_port); + + /*if (materializeMySQLDatabase(define->settings)) + return std::make_shared( + context, database_name, metadata_path, define, mysql_database_name, std::move(mysql_pool));*/ + + return std::make_shared(context, database_name, metadata_path, define, mysql_database_name, std::move(mysql_pool)); + } + catch (...) + { + const auto & exception_message = getCurrentExceptionMessage(true); + throw Exception("Cannot create MySQL database, because " + exception_message, ErrorCodes::CANNOT_CREATE_DATABASE); + } +} + +} + +#endif diff --git a/src/Databases/MySQL/createMySQLDatabase.h b/src/Databases/MySQL/createMySQLDatabase.h new file mode 100644 index 00000000000..f85e96428fe --- /dev/null +++ b/src/Databases/MySQL/createMySQLDatabase.h @@ -0,0 +1,19 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#if USE_MYSQL + +#include +#include + +namespace DB +{ + +DatabasePtr createMySQLDatabase(const String & database_name, const String & metadata_path, const ASTStorage * define, Context & context); + +} + +#endif diff --git a/src/Parsers/CMakeLists.txt b/src/Parsers/CMakeLists.txt index b83fc20e818..13e460da4e4 100644 --- a/src/Parsers/CMakeLists.txt +++ b/src/Parsers/CMakeLists.txt @@ -1,5 +1,6 @@ include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) add_headers_and_sources(clickhouse_parsers .) +add_headers_and_sources(clickhouse_parsers ./MySQL) add_library(clickhouse_parsers ${clickhouse_parsers_headers} ${clickhouse_parsers_sources}) target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io) diff --git a/src/Parsers/MySQL/ASTCreateDefines.cpp b/src/Parsers/MySQL/ASTCreateDefines.cpp new file mode 100644 index 00000000000..c01602b05c6 --- /dev/null +++ b/src/Parsers/MySQL/ASTCreateDefines.cpp @@ -0,0 +1,95 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ParserCreateDefine : public IParserBase +{ +protected: + + const char * getName() const override { return "table property (column, index, constraint)"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + MySQLParser::ParserDeclareIndex p_declare_index; + MySQLParser::ParserDeclareColumn p_declare_column; + MySQLParser::ParserDeclareConstraint p_declare_constraint; + + if (likely(!p_declare_index.parse(pos, node, expected))) + { + if (likely(!p_declare_constraint.parse(pos, node, expected))) + { + if (!p_declare_column.parse(pos, node, expected)) + return false; + } + } + + return true; + } +}; + +ASTPtr ASTCreateDefines::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (columns) + res->set(res->columns, columns->clone()); + + if (indices) + res->set(res->indices, indices->clone()); + + if (constraints) + res->set(res->constraints, constraints->clone()); + + return res; +} + +bool ParserCreateDefines::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr create_defines; + ParserList create_defines_parser(std::make_unique(), std::make_unique(TokenType::Comma), false); + + if (!create_defines_parser.parse(pos, create_defines, expected)) + return false; + + ASTPtr columns = std::make_shared(); + ASTPtr indices = std::make_shared(); + ASTPtr constraints = std::make_shared(); + + for (const auto & create_define : create_defines->children) + { + if (create_define->as()) + columns->children.push_back(create_define); + else if (create_define->as()) + indices->children.push_back(create_define); + else if (create_define->as()) + constraints->children.push_back(create_define); + else + return false; + } + + auto res = std::make_shared(); + if (!columns->children.empty()) + res->set(res->columns, columns); + if (!indices->children.empty()) + res->set(res->indices, indices); + if (!constraints->children.empty()) + res->set(res->constraints, constraints); + + node = res; + return true; +} + +} + +} + diff --git a/src/Parsers/MySQL/ASTCreateDefines.h b/src/Parsers/MySQL/ASTCreateDefines.h new file mode 100644 index 00000000000..72adb1c745a --- /dev/null +++ b/src/Parsers/MySQL/ASTCreateDefines.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + + +class ASTCreateDefines : public IAST +{ +public: + ASTExpressionList * columns = nullptr; + ASTExpressionList * indices = nullptr; + ASTExpressionList * constraints = nullptr; + + ASTPtr clone() const override; + + String getID(char) const override { return "Create definitions"; } +}; + +class ParserCreateDefines : public IParserBase +{ +protected: + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + + const char * getName() const override { return "table property list (column, index, constraint)"; } +}; + +} + +} + diff --git a/src/Parsers/MySQL/ASTCreateQuery.cpp b/src/Parsers/MySQL/ASTCreateQuery.cpp new file mode 100644 index 00000000000..a200ce2e0ff --- /dev/null +++ b/src/Parsers/MySQL/ASTCreateQuery.cpp @@ -0,0 +1,128 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +ASTPtr ASTCreateQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (columns_list) + { + res->columns_list = columns_list->clone(); + res->children.emplace_back(res->columns_list); + } + + if (table_options) + { + res->table_options = table_options->clone(); + res->children.emplace_back(res->table_options); + } + + if (partition_options) + { + res->partition_options = partition_options->clone(); + res->children.emplace_back(res->partition_options); + } + + return res; +} + +bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr table; + ASTPtr like_table; + ASTPtr columns_list; + ASTPtr table_options; + ASTPtr partition_options; + bool is_temporary = false; + bool if_not_exists = false; + + if (!ParserKeyword("CREATE").ignore(pos, expected)) + return false; + + if (ParserKeyword("TEMPORARY").ignore(pos, expected)) + is_temporary = true; + + if (!ParserKeyword("TABLE").ignore(pos, expected)) + return false; + + if (ParserKeyword("IF NOT EXISTS").ignore(pos, expected)) + if_not_exists = true; + + if (!ParserCompoundIdentifier(true).parse(pos, table, expected)) + return false; + + if (ParserKeyword("LIKE").ignore(pos, expected)) + { + if (!ParserCompoundIdentifier(true).parse(pos, like_table, expected)) + return false; + } + + if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)) + { + if (ParserKeyword("LIKE").ignore(pos, expected)) + { + if (!ParserCompoundIdentifier(true).parse(pos, like_table, expected)) + return false; + + if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + return false; + } + else + { + if (!ParserCreateDefines().parse(pos, columns_list, expected)) + return false; + + if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + return false; + + ParserDeclareTableOptions().parse(pos, table_options, expected); + ParserDeclarePartitionOptions().parse(pos, partition_options, expected); + } + } + + auto create_query = std::make_shared(); + + create_query->temporary = is_temporary; + create_query->if_not_exists = if_not_exists; + + StorageID table_id = getTableIdentifier(table); + create_query->table = table_id.table_name; + create_query->database = table_id.database_name; + create_query->like_table = like_table; + create_query->columns_list = columns_list; + create_query->table_options = table_options; + create_query->partition_options = partition_options; + + if (create_query->like_table) + create_query->children.emplace_back(create_query->like_table); + + if (create_query->columns_list) + create_query->children.emplace_back(create_query->columns_list); + + if (create_query->table_options) + create_query->children.emplace_back(create_query->table_options); + + if (create_query->partition_options) + create_query->children.emplace_back(create_query->partition_options); + + node = create_query; + return true; +} +} + +} diff --git a/src/Parsers/MySQL/ASTCreateQuery.h b/src/Parsers/MySQL/ASTCreateQuery.h new file mode 100644 index 00000000000..2d1deb23d4b --- /dev/null +++ b/src/Parsers/MySQL/ASTCreateQuery.h @@ -0,0 +1,40 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTCreateQuery : public IAST +{ +public: + bool temporary{false}; + bool if_not_exists{false}; + + String table; + String database; + ASTPtr like_table; + ASTPtr columns_list; + ASTPtr table_options; + ASTPtr partition_options; + + ASTPtr clone() const override; + + String getID(char) const override { return "create query"; } +}; + +class ParserCreateQuery : public IParserBase +{ +protected: + const char * getName() const override { return "create query"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareColumn.cpp b/src/Parsers/MySQL/ASTDeclareColumn.cpp new file mode 100644 index 00000000000..295818a66c5 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareColumn.cpp @@ -0,0 +1,98 @@ +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +ASTPtr ASTDeclareColumn::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (data_type) + { + res->data_type = data_type->clone(); + res->children.emplace_back(res->data_type); + } + + if (column_options) + { + res->column_options = column_options->clone(); + res->children.emplace_back(res->column_options); + } + + return res; +} + +bool ParserDeclareColumn::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr column_name; + ASTPtr column_data_type; + ASTPtr column_options; + + ParserExpression p_expression; + ParserIdentifier p_identifier; + + if (!p_identifier.parse(pos, column_name, expected)) + return false; + + if (!p_expression.parse(pos, column_data_type, expected)) + return false; + + if (!parseColumnDeclareOptions(pos, column_options, expected)) + return false; + + auto declare_column = std::make_shared(); + declare_column->name = column_name->as()->name; + declare_column->data_type = column_data_type; + declare_column->column_options = column_options; + + if (declare_column->data_type) + declare_column->children.emplace_back(declare_column->data_type); + + if (declare_column->column_options) + declare_column->children.emplace_back(declare_column->column_options); + + node = declare_column; + return true; +} +bool ParserDeclareColumn::parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserDeclareOption p_non_generate_options{ + { + OptionDescribe("NULL", "is_null", std::make_unique()), + OptionDescribe("NOT NULL", "is_null", std::make_unique()), + OptionDescribe("DEFAULT", "default", std::make_unique()), + OptionDescribe("AUTO_INCREMENT", "auto_increment", std::make_unique()), + OptionDescribe("UNIQUE", "unique_key", std::make_unique()), + OptionDescribe("UNIQUE KEY", "unique_key", std::make_unique()), + OptionDescribe("KEY", "primary_key", std::make_unique()), + OptionDescribe("PRIMARY KEY", "primary_key", std::make_unique()), + OptionDescribe("COMMENT", "comment", std::make_unique()), + OptionDescribe("COLLATE", "collate", std::make_unique()), + OptionDescribe("COLUMN_FORMAT", "column_format", std::make_unique()), + OptionDescribe("STORAGE", "storage", std::make_unique()), + OptionDescribe("AS", "generated", std::make_unique()), + OptionDescribe("GENERATED ALWAYS AS", "generated", std::make_unique()), + OptionDescribe("STORED", "is_stored", std::make_unique()), + OptionDescribe("VIRTUAL", "is_stored", std::make_unique()), + OptionDescribe("", "reference", std::make_unique()), + OptionDescribe("", "constraint", std::make_unique()), + } + }; + + return p_non_generate_options.parse(pos, node, expected); +} + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareColumn.h b/src/Parsers/MySQL/ASTDeclareColumn.h new file mode 100644 index 00000000000..25103072e71 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareColumn.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTDeclareColumn : public IAST +{ +public: + String name; + ASTPtr data_type; + ASTPtr column_options; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "Column definition"; } +}; + +class ParserDeclareColumn : public IParserBase +{ +protected: + const char * getName() const override { return "index declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + + bool parseColumnDeclareOptions(Pos & pos, ASTPtr & node, Expected & expected); +}; + + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareConstraint.cpp b/src/Parsers/MySQL/ASTDeclareConstraint.cpp new file mode 100644 index 00000000000..0f447fb3b40 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareConstraint.cpp @@ -0,0 +1,74 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +ASTPtr ASTDeclareConstraint::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (check_expression) + { + res->check_expression = check_expression->clone(); + res->children.emplace_back(res->check_expression); + } + + return res; +} + +bool ParserDeclareConstraint::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + bool enforced = true; + ASTPtr constraint_symbol; + ASTPtr index_check_expression; + ParserExpression p_expression; + + if (ParserKeyword("CONSTRAINT").ignore(pos, expected)) + { + if (!ParserKeyword("CHECK").checkWithoutMoving(pos, expected)) + ParserIdentifier().parse(pos, constraint_symbol, expected); + } + + + if (!ParserKeyword("CHECK").ignore(pos, expected)) + return false; + + if (!p_expression.parse(pos, index_check_expression, expected)) + return false; + + if (ParserKeyword("NOT").ignore(pos, expected)) + { + if (!ParserKeyword("ENFORCED").ignore(pos, expected)) + return false; + + enforced = false; + } + else + { + enforced = true; + ParserKeyword("ENFORCED").ignore(pos, expected); + } + + auto declare_constraint = std::make_shared(); + declare_constraint->enforced = enforced; + declare_constraint->check_expression = index_check_expression; + + if (constraint_symbol) + declare_constraint->constraint_name = constraint_symbol->as()->name; + + node = declare_constraint; + return true; +} + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareConstraint.h b/src/Parsers/MySQL/ASTDeclareConstraint.h new file mode 100644 index 00000000000..6af40e499eb --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareConstraint.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTDeclareConstraint : public IAST +{ +public: + bool enforced{true}; + String constraint_name; + ASTPtr check_expression; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "constraint declaration"; } +}; + +class ParserDeclareConstraint : public IParserBase +{ +protected: + const char * getName() const override { return "constraint declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} + +} + diff --git a/src/Parsers/MySQL/ASTDeclareIndex.cpp b/src/Parsers/MySQL/ASTDeclareIndex.cpp new file mode 100644 index 00000000000..592e01e23a3 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareIndex.cpp @@ -0,0 +1,235 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +struct ParserIndexColumn : public IParserBase +{ +protected: + const char * getName() const override { return "index column"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + ParserExpression p_expression; + + if (!p_expression.parse(pos, node, expected)) + return false; + + ParserKeyword("ASC").ignore(pos, expected); + ParserKeyword("DESC").ignore(pos, expected); + return true; + } +}; + +ASTPtr ASTDeclareIndex::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (index_columns) + { + res->index_columns = index_columns->clone(); + res->children.emplace_back(res->index_columns); + } + + + if (index_options) + { + res->index_options = index_options->clone(); + res->children.emplace_back(res->index_options); + } + + + if (reference_definition) + { + res->reference_definition = reference_definition->clone(); + res->children.emplace_back(res->reference_definition); + } + + return res; +} +bool ParserDeclareIndex::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + String index_name; + String index_type; + ASTPtr index_columns; + ASTPtr index_options; + ASTPtr declare_reference; + ParserIndexColumn p_expression; + + ParserDeclareOption p_index_options{ + { + OptionDescribe("KEY_BLOCK_SIZE", "key_block_size", std::make_unique()), + OptionDescribe("USING", "index_type", std::make_unique()), + OptionDescribe("WITH PARSER", "index_parser", std::make_unique()), + OptionDescribe("COMMENT", "comment", std::make_unique()), + OptionDescribe("VISIBLE", "visible", std::make_unique()), + OptionDescribe("INVISIBLE", "visible", std::make_unique()), + } + }; + + if (!parseDeclareOrdinaryIndex(pos, index_name, index_type, expected)) + { + if (!parseDeclareConstraintIndex(pos, index_name, index_type, expected)) + return false; + } + + ParserToken s_opening_round(TokenType::OpeningRoundBracket); + ParserToken s_closing_round(TokenType::ClosingRoundBracket); + + if (!s_opening_round.ignore(pos, expected)) + return false; + + ParserList p_index_columns(std::make_unique(), std::make_unique(TokenType::Comma)); + + if (!p_index_columns.parse(pos, index_columns, expected)) + return false; + + if (!s_closing_round.ignore(pos, expected)) + return false; + + if (index_type != "FOREIGN") + p_index_options.parse(pos, index_options, expected); + else + { + if (!ParserDeclareReference().parse(pos, declare_reference, expected)) + return false; + } + + auto declare_index = std::make_shared(); + declare_index->index_name = index_name; + declare_index->index_type = index_type; + declare_index->index_columns = index_columns; + declare_index->index_options = index_options; + declare_index->reference_definition = declare_reference; + + if (declare_index->index_columns) + declare_index->children.emplace_back(declare_index->index_columns); + + if (declare_index->index_options) + declare_index->children.emplace_back(declare_index->index_options); + + if (declare_index->reference_definition) + declare_index->children.emplace_back(declare_index->reference_definition); + + node = declare_index; + return true; +} +bool ParserDeclareIndex::parseDeclareOrdinaryIndex(IParser::Pos & pos, String & index_name, String & index_type, Expected & expected) +{ + ASTPtr temp_node; + ParserKeyword k_key("KEY"); + ParserKeyword k_index("INDEX"); + + ParserExpression p_expression; + ParserIdentifier p_identifier; + + if (ParserKeyword("SPATIAL").ignore(pos, expected)) + { + if (!k_key.ignore(pos, expected)) + k_index.ignore(pos, expected); + + index_type = "SPATIAL"; + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; + } + else if (ParserKeyword("FULLTEXT").ignore(pos, expected)) + { + if (!k_key.ignore(pos, expected)) + k_index.ignore(pos, expected); + + index_type = "FULLTEXT"; + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; + } + else + { + if (!k_key.ignore(pos, expected)) + { + if (!k_index.ignore(pos, expected)) + return false; + } + + index_type = "BTREE"; /// default index type + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; + + if (ParserKeyword("USING").ignore(pos, expected)) + { + if (!p_identifier.parse(pos, temp_node, expected)) + return false; + + index_type = temp_node->as()->name; + } + } + + return true; +} + +bool ParserDeclareIndex::parseDeclareConstraintIndex(IParser::Pos & pos, String & index_name, String & index_type, Expected & expected) +{ + ASTPtr temp_node; + ParserIdentifier p_identifier; + + if (ParserKeyword("CONSTRAINT").ignore(pos, expected)) + { + + if (!ParserKeyword("PRIMARY").checkWithoutMoving(pos, expected) && !ParserKeyword("UNIQUE").checkWithoutMoving(pos, expected) + && !ParserKeyword("FOREIGN").checkWithoutMoving(pos, expected)) + { + if (!p_identifier.parse(pos, temp_node, expected)) + return false; + + index_name = temp_node->as()->name; + } + } + + if (ParserKeyword("UNIQUE").ignore(pos, expected)) + { + if (!ParserKeyword("KEY").ignore(pos, expected)) + ParserKeyword("INDEX").ignore(pos, expected); + + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; /// reset index_name + + index_type = "UNIQUE_BTREE"; /// default btree index_type + if (ParserKeyword("USING").ignore(pos, expected)) + { + if (!p_identifier.parse(pos, temp_node, expected)) + return false; + + index_type = temp_node->as()->name; + } + } + else if (ParserKeyword("PRIMARY KEY").ignore(pos, expected)) + { + index_type = "PRIMARY_KEY_BTREE"; /// default btree index_type + if (ParserKeyword("USING").ignore(pos, expected)) + { + if (!p_identifier.parse(pos, temp_node, expected)) + return false; + + index_type = temp_node->as()->name; + } + } + else if (ParserKeyword("FOREIGN KEY").ignore(pos, expected)) + { + index_type = "FOREIGN"; + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; /// reset index_name + } + + return true; +} +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareIndex.h b/src/Parsers/MySQL/ASTDeclareIndex.h new file mode 100644 index 00000000000..d3dc3741c0f --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareIndex.h @@ -0,0 +1,43 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTDeclareIndex: public IAST +{ +public: + String index_name; + String index_type; + ASTPtr index_columns; + ASTPtr index_options; + ASTPtr reference_definition; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "index declaration"; } +}; + +class ParserDeclareIndex : public IParserBase +{ +protected: + const char * getName() const override { return "index declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + + bool parseDeclareOrdinaryIndex(Pos & pos, String & index_name, String & index_type, Expected & expected); + + bool parseDeclareConstraintIndex(Pos & pos, String & index_name, String & index_type, Expected & expected); + +}; + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareOption.cpp b/src/Parsers/MySQL/ASTDeclareOption.cpp new file mode 100644 index 00000000000..347fd5d479b --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareOption.cpp @@ -0,0 +1,142 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +bool ParserDeclareOption::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + std::unordered_map changes; + std::unordered_map> usage_parsers_cached; + usage_parsers_cached.reserve(options_collection.size()); + + const auto & get_parser_from_cache = [&](const char * usage_name) + { + auto iterator = usage_parsers_cached.find(usage_name); + if (iterator == usage_parsers_cached.end()) + iterator = usage_parsers_cached.insert(std::make_pair(usage_name, std::make_shared(usage_name))).first; + + return iterator->second; + }; + + while (true) + { + ASTPtr value; + bool found{false}; + for (const auto & option_describe : options_collection) + { + if (strlen(option_describe.usage_name) == 0) + { + if (option_describe.value_parser->parse(pos, value, expected)) + { + found = true; + changes.insert(std::make_pair(option_describe.option_name, value)); + } + } + else if (get_parser_from_cache(option_describe.usage_name)->ignore(pos, expected)) + { + ParserToken{TokenType::Equals}.ignore(pos, expected); + + if (!option_describe.value_parser->parse(pos, value, expected)) + return false; + + /*const auto & changes_iterator = changes.find(option_describe.option_name); + if (changes_iterator != changes.end()) + throw Exception("Duplicate options declare", ErrorCodes::)*/ + found = true; + changes.insert(std::make_pair(option_describe.option_name, value)); + } + } + + if (!found) + break; + + ParserToken{TokenType::Comma}.ignore(pos, expected); + } + + if (!changes.empty()) + { + auto options_declare = std::make_shared(); + options_declare->changes = changes; + + node = options_declare; + } + + return true; +} + +ASTPtr ASTDeclareOptions::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + res->changes.clear(); + + for (const auto & [name, value] : this->changes) + res->changes.insert(std::make_pair(name, value->clone())); + + return res; +} + +bool ParserAlwaysTrue::parseImpl(IParser::Pos & /*pos*/, ASTPtr & node, Expected & /*expected*/) +{ + node = std::make_shared(Field(UInt64(1))); + return true; +} + +bool ParserAlwaysFalse::parseImpl(IParser::Pos & /*pos*/, ASTPtr & node, Expected & /*expected*/) +{ + node = std::make_shared(Field(UInt64(0))); + return true; +} + +bool ParserCharsetName::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &) +{ + /// Identifier in backquotes or in double quotes + if (pos->type == TokenType::QuotedIdentifier) + { + ReadBufferFromMemory buf(pos->begin, pos->size()); + String s; + + if (*pos->begin == '`') + readBackQuotedStringWithSQLStyle(s, buf); + else + readDoubleQuotedStringWithSQLStyle(s, buf); + + if (s.empty()) /// Identifiers "empty string" are not allowed. + return false; + + node = std::make_shared(s); + ++pos; + return true; + } + else if (pos->type == TokenType::BareWord) + { + const char * begin = pos->begin; + + while (true) + { + if (isWhitespaceASCII(*pos->end)) + break; + else + { + ++pos; + } + } + + node = std::make_shared(String(begin, pos->end)); + ++pos; + return true; + } + + return false; +} +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareOption.h b/src/Parsers/MySQL/ASTDeclareOption.h new file mode 100644 index 00000000000..fa6c25bd914 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareOption.h @@ -0,0 +1,76 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +struct OptionDescribe +{ + const char * usage_name; + String option_name; + std::shared_ptr value_parser; + + OptionDescribe(const char * usage_name_, const String & option_name_, const std::shared_ptr & value_parser_) + :usage_name(usage_name_), option_name(option_name_), value_parser(value_parser_) + { + } +}; + +class ASTDeclareOptions : public IAST +{ +public: + std::unordered_map changes; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "options declaration"; } +}; + +class ParserAlwaysTrue : public IParserBase +{ +public: + const char * getName() const override { return "always true"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +class ParserAlwaysFalse : public IParserBase +{ +public: + const char * getName() const override { return "always false"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +/// Copy and paste from ParserIdentifier, +/// the difference is that multiple tokens are glued if there is no whitespace ASCII between them +struct ParserCharsetName : public IParserBase +{ +protected: + const char * getName() const override { return "charset name"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected &) override; +}; + +class ParserDeclareOption : public IParserBase +{ +protected: + std::vector options_collection; + + const char * getName() const override { return "option declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +public: + ParserDeclareOption(const std::vector & options_collection_) : options_collection(options_collection_) {} +}; + +} + + +} diff --git a/src/Parsers/MySQL/ASTDeclarePartition.cpp b/src/Parsers/MySQL/ASTDeclarePartition.cpp new file mode 100644 index 00000000000..68a56109e64 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclarePartition.cpp @@ -0,0 +1,127 @@ +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +ASTPtr ASTDeclarePartition::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (options) + { + res->options = options->clone(); + res->children.emplace_back(res->options); + } + + if (less_than) + { + res->less_than = less_than->clone(); + res->children.emplace_back(res->less_than); + } + + if (in_expression) + { + res->in_expression = in_expression->clone(); + res->children.emplace_back(res->in_expression); + } + + if (subpartitions) + { + res->subpartitions = subpartitions->clone(); + res->children.emplace_back(res->subpartitions); + } + + return res; +} + +bool ParserDeclarePartition::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + if (!ParserKeyword{"PARTITION"}.ignore(pos, expected)) + return false; + + ASTPtr options; + ASTPtr less_than; + ASTPtr in_expression; + ASTPtr partition_name; + + ParserExpression p_expression; + ParserIdentifier p_identifier; + + if (!p_identifier.parse(pos, partition_name, expected)) + return false; + + ParserKeyword p_values("VALUES"); + if (p_values.ignore(pos, expected)) + { + if (ParserKeyword{"IN"}.ignore(pos, expected)) + { + if (!p_expression.parse(pos, in_expression, expected)) + return false; + } + else if (ParserKeyword{"LESS THAN"}.ignore(pos, expected)) + { + if (!p_expression.parse(pos, less_than, expected)) + return false; + } + } + + if (!ParserDeclareOption{ + { + OptionDescribe("ENGINE", "engine", std::make_shared()), + OptionDescribe("STORAGE ENGINE", "engine", std::make_shared()), + OptionDescribe("COMMENT", "comment", std::make_shared()), + OptionDescribe("DATA DIRECTORY", "data_directory", std::make_shared()), + OptionDescribe("INDEX DIRECTORY", "index_directory", std::make_shared()), + OptionDescribe("MAX_ROWS", "max_rows", std::make_shared()), + OptionDescribe("MIN_ROWS", "min_rows", std::make_shared()), + OptionDescribe("TABLESPACE", "tablespace", std::make_shared()), + } + }.parse(pos, options, expected)) + return false; + + ASTPtr subpartitions; + if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)) + { + if (!DB::ParserList(std::make_unique(), std::make_unique(TokenType::Comma)) + .parse(pos, subpartitions, expected)) + return false; + + if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + return false; + } + + auto partition_declare = std::make_shared(); + partition_declare->options = options; + partition_declare->less_than = less_than; + partition_declare->in_expression = in_expression; + partition_declare->subpartitions = subpartitions; + partition_declare->partition_name = partition_name->as()->name; + + if (partition_declare->options) + partition_declare->children.emplace_back(partition_declare->options); + + if (partition_declare->less_than) + partition_declare->children.emplace_back(partition_declare->less_than); + + if (partition_declare->in_expression) + partition_declare->children.emplace_back(partition_declare->in_expression); + + if (partition_declare->subpartitions) + partition_declare->children.emplace_back(partition_declare->subpartitions); + + node = partition_declare; + return true; +} +} + +} diff --git a/src/Parsers/MySQL/ASTDeclarePartition.h b/src/Parsers/MySQL/ASTDeclarePartition.h new file mode 100644 index 00000000000..ef076dcd6cf --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclarePartition.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTDeclarePartition : public IAST +{ +public: + String partition_name; + ASTPtr less_than; + ASTPtr in_expression; + ASTPtr options; + ASTPtr subpartitions; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "partition declaration"; } +}; + +class ParserDeclarePartition : public IParserBase +{ +protected: + const char * getName() const override { return "partition declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} + +} diff --git a/src/Parsers/MySQL/ASTDeclarePartitionOptions.cpp b/src/Parsers/MySQL/ASTDeclarePartitionOptions.cpp new file mode 100644 index 00000000000..ff1ffb93f0c --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclarePartitionOptions.cpp @@ -0,0 +1,184 @@ +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +ASTPtr ASTDeclarePartitionOptions::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (partition_numbers) + { + res->partition_numbers = partition_numbers->clone(); + res->children.emplace_back(res->partition_numbers); + } + + if (partition_expression) + { + res->partition_expression = partition_expression->clone(); + res->children.emplace_back(res->partition_expression); + } + + if (subpartition_numbers) + { + res->subpartition_numbers = subpartition_numbers->clone(); + res->children.emplace_back(res->subpartition_numbers); + } + + if (subpartition_expression) + { + res->subpartition_expression = subpartition_expression->clone(); + res->children.emplace_back(res->subpartition_expression); + } + + return res; +} + +bool ParserDeclarePartitionOptions::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + String partition_type; + ASTPtr partition_numbers; + ASTPtr partition_expression; + String subpartition_type; + ASTPtr subpartition_numbers; + ASTPtr subpartition_expression; + ASTPtr declare_partitions; + + if (!ParserKeyword("PARTITION BY").ignore(pos, expected)) + return false; + + if (!parsePartitionExpression(pos, partition_type, partition_expression, expected)) + return false; + + if (ParserKeyword("PARTITIONS").ignore(pos, expected)) + { + ParserLiteral p_literal; + if (!p_literal.parse(pos, partition_numbers, expected)) + return false; + } + + if (ParserKeyword("SUBPARTITION BY").ignore(pos, expected)) + { + if (!parsePartitionExpression(pos, subpartition_type, subpartition_expression, expected, true)) + return false; + + if (ParserKeyword("SUBPARTITIONS").ignore(pos, expected)) + { + ParserLiteral p_literal; + if (!p_literal.parse(pos, subpartition_numbers, expected)) + return false; + } + } + + if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)) + { + if (!ParserList(std::make_unique(), std::make_unique(TokenType::Comma)) + .parse(pos, declare_partitions, expected)) + return false; + + if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + return false; + } + + auto declare_partition_options = std::make_shared(); + declare_partition_options->partition_type = partition_type; + declare_partition_options->partition_numbers = partition_numbers; + declare_partition_options->partition_expression = partition_expression; + declare_partition_options->subpartition_type = subpartition_type; + declare_partition_options->subpartition_numbers = subpartition_numbers; + declare_partition_options->subpartition_expression = subpartition_expression; + declare_partition_options->declare_partitions = declare_partitions; + + if (declare_partition_options->partition_numbers) + declare_partition_options->children.emplace_back(declare_partition_options->partition_numbers); + + if (declare_partition_options->partition_expression) + declare_partition_options->children.emplace_back(declare_partition_options->partition_expression); + + if (declare_partition_options->subpartition_numbers) + declare_partition_options->children.emplace_back(declare_partition_options->subpartition_numbers); + + if (declare_partition_options->subpartition_expression) + declare_partition_options->children.emplace_back(declare_partition_options->subpartition_expression); + + if (declare_partition_options->declare_partitions) + declare_partition_options->children.emplace_back(declare_partition_options->declare_partitions); + + node = declare_partition_options; + return true; +} + +bool ParserDeclarePartitionOptions::parsePartitionExpression(Pos & pos, std::string & type, ASTPtr & node, Expected & expected, bool subpartition) +{ + ASTPtr expression; + ParserExpression p_expression; + if (!subpartition && ParserKeyword("LIST").ignore(pos, expected)) + { + type = "list"; + ParserKeyword("COLUMNS").ignore(pos, expected); + if (!p_expression.parse(pos, expression, expected)) + return false; + } + else if (!subpartition && ParserKeyword("RANGE").ignore(pos, expected)) + { + type = "range"; + ParserKeyword("COLUMNS").ignore(pos, expected); + if (!p_expression.parse(pos, expression, expected)) + return false; + } + else + { + if (ParserKeyword("LINEAR").ignore(pos, expected)) + type = "linear_"; + + if (ParserKeyword("KEY").ignore(pos, expected)) + { + type += "key"; + + if (ParserKeyword("ALGORITHM").ignore(pos, expected)) + { + if (!ParserToken(TokenType::Equals).ignore(pos, expected)) + return false; + + ASTPtr algorithm; + ParserLiteral p_literal; + if (!p_literal.parse(pos, algorithm, expected) || !algorithm->as()) + return false; + + UInt64 algorithm_type = algorithm->as()->value.safeGet(); + + if (algorithm_type != 1 && algorithm_type != 2) + return false; + + type += "_" + toString(algorithm_type); + } + + if (!p_expression.parse(pos, expression, expected)) + return false; + } + else if (ParserKeyword("HASH").ignore(pos, expected)) + { + type += "hash"; + if (!p_expression.parse(pos, expression, expected)) + return false; + } + else + return false; + } + + node = expression; + return true; +} +} + +} diff --git a/src/Parsers/MySQL/ASTDeclarePartitionOptions.h b/src/Parsers/MySQL/ASTDeclarePartitionOptions.h new file mode 100644 index 00000000000..f560c66534e --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclarePartitionOptions.h @@ -0,0 +1,40 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTDeclarePartitionOptions : public IAST +{ +public: + String partition_type; + ASTPtr partition_numbers; + ASTPtr partition_expression; + String subpartition_type; + ASTPtr subpartition_numbers; + ASTPtr subpartition_expression; + ASTPtr declare_partitions; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "partition options declaration"; } +}; + +class ParserDeclarePartitionOptions : public IParserBase +{ +protected: + const char * getName() const override { return "partition options declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + + bool parsePartitionExpression(Pos & pos, std::string & type, ASTPtr & node, Expected & expected, bool subpartition = false); +}; + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareReference.cpp b/src/Parsers/MySQL/ASTDeclareReference.cpp new file mode 100644 index 00000000000..434b9561eda --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareReference.cpp @@ -0,0 +1,105 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +bool parseReferenceOption(IParser::Pos & pos, ASTDeclareReference::ReferenceOption & option, Expected & expected) +{ + if (ParserKeyword("RESTRICT").ignore(pos, expected)) + option = ASTDeclareReference::RESTRICT; + else if (ParserKeyword("CASCADE").ignore(pos, expected)) + option = ASTDeclareReference::CASCADE; + else if (ParserKeyword("SET NULL").ignore(pos, expected)) + option = ASTDeclareReference::SET_NULL; + else if (ParserKeyword("NO ACTION").ignore(pos, expected)) + option = ASTDeclareReference::NO_ACTION; + else if (ParserKeyword("SET DEFAULT").ignore(pos, expected)) + option = ASTDeclareReference::SET_DEFAULT; + else + return false; + + return true; +} + +ASTPtr ASTDeclareReference::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (reference_expression) + { + res->reference_expression = reference_expression->clone(); + res->children.emplace_back(res->reference_expression); + } + + return res; +} + +bool ParserDeclareReference::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr table_name; + ASTPtr expression; + ParserExpression p_expression; + ParserIdentifier p_identifier; + ASTDeclareReference::MatchKind match_kind = ASTDeclareReference::MATCH_FULL; + ASTDeclareReference::ReferenceOption delete_option = ASTDeclareReference::RESTRICT; + ASTDeclareReference::ReferenceOption update_option = ASTDeclareReference::RESTRICT; + + if (!ParserKeyword("REFERENCES").ignore(pos, expected)) + return false; + + if (!p_identifier.parse(pos, table_name, expected)) + return false; + + if (!p_expression.parse(pos, expression, expected)) + return false; + + if (ParserKeyword("MATCH").ignore(pos, expected)) + { + if (ParserKeyword("FULL").ignore(pos, expected)) + match_kind = ASTDeclareReference::MATCH_FULL; + else if (ParserKeyword("SIMPLE").ignore(pos, expected)) + match_kind = ASTDeclareReference::MATCH_SIMPLE; + else if (ParserKeyword("PARTIAL").ignore(pos, expected)) + match_kind = ASTDeclareReference::MATCH_PARTIAL; + else + return false; + } + + while (true) + { + if (ParserKeyword("ON DELETE").ignore(pos, expected)) + { + if (!parseReferenceOption(pos, delete_option, expected)) + return false; + } + else if (ParserKeyword("ON UPDATE").ignore(pos, expected)) + { + if (!parseReferenceOption(pos, update_option, expected)) + return false; + } + else + break; + } + + auto declare_reference = std::make_shared(); + declare_reference->kind = match_kind; + declare_reference->on_delete_option = delete_option; + declare_reference->on_update_option = update_option; + declare_reference->reference_expression = expression; + declare_reference->reference_table_name = table_name->as()->name; + + node = declare_reference; + return true; +} +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareReference.h b/src/Parsers/MySQL/ASTDeclareReference.h new file mode 100644 index 00000000000..41cd6597dfb --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareReference.h @@ -0,0 +1,52 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTDeclareReference : public IAST +{ +public: + enum MatchKind + { + MATCH_FULL, + MATCH_PARTIAL, + MATCH_SIMPLE + }; + + enum ReferenceOption + { + RESTRICT, + CASCADE, + SET_NULL, + NO_ACTION, + SET_DEFAULT + }; + + MatchKind kind; + String reference_table_name; + ASTPtr reference_expression; + ReferenceOption on_delete_option; + ReferenceOption on_update_option; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "subpartition declaration"; } +}; + +class ParserDeclareReference : public IParserBase +{ +protected: + const char * getName() const override { return "reference declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareSubPartition.cpp b/src/Parsers/MySQL/ASTDeclareSubPartition.cpp new file mode 100644 index 00000000000..25c04779656 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareSubPartition.cpp @@ -0,0 +1,66 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +bool ParserDeclareSubPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (!ParserKeyword{"SUBPARTITION"}.ignore(pos, expected)) + return false; + + ASTPtr options; + ASTPtr logical_name; + ParserIdentifier p_identifier; + + if (!p_identifier.parse(pos, logical_name, expected)) + return false; + + if (!ParserDeclareOption{ + { + OptionDescribe("ENGINE", "engine", std::make_shared()), + OptionDescribe("STORAGE ENGINE", "engine", std::make_shared()), + OptionDescribe("COMMENT", "comment", std::make_shared()), + OptionDescribe("DATA DIRECTORY", "data_directory", std::make_shared()), + OptionDescribe("INDEX DIRECTORY", "index_directory", std::make_shared()), + OptionDescribe("MAX_ROWS", "max_rows", std::make_shared()), + OptionDescribe("MIN_ROWS", "min_rows", std::make_shared()), + OptionDescribe("TABLESPACE", "tablespace", std::make_shared()), + } + }.parse(pos, options, expected)) + return false; + + auto subpartition_declare = std::make_shared(); + subpartition_declare->options = options; + subpartition_declare->logical_name = logical_name->as()->name; + + if (subpartition_declare->options) + subpartition_declare->children.emplace_back(subpartition_declare->options); + + node = subpartition_declare; + return true; +} + +ASTPtr ASTDeclareSubPartition::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (options) + { + res->options = options->clone(); + res->children.emplace_back(res->options); + } + + return res; +} +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareSubPartition.h b/src/Parsers/MySQL/ASTDeclareSubPartition.h new file mode 100644 index 00000000000..018cf2c8c4e --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareSubPartition.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTDeclareSubPartition : public IAST +{ +public: + ASTPtr options; + String logical_name; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "subpartition declaration"; } +}; + +class ParserDeclareSubPartition : public IParserBase +{ +protected: + const char * getName() const override { return "subpartition declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareTableOptions.cpp b/src/Parsers/MySQL/ASTDeclareTableOptions.cpp new file mode 100644 index 00000000000..066e8c4c414 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareTableOptions.cpp @@ -0,0 +1,102 @@ +#include + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +template +struct ParserBoolOption : public IParserBase +{ +protected: + const char * getName() const override { return "bool option with default"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + if constexpr(allow_default) + { + if (ParserKeyword("DEFAULT").ignore(pos, expected)) + { + node = std::make_shared("DEFAULT"); + return true; + } + } + ParserLiteral p_literal; + if (!p_literal.parse(pos, node, expected) || !node->as()) + return false; + + return !(node->as()->value.safeGet() != 0 && node->as()->value.safeGet() != 1); + } +}; + +struct ParserTablespaceName : public IParserBase +{ +protected: + const char * getName() const override { return "table space name"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + ParserIdentifier p_identifier; + if (!p_identifier.parse(pos, node, expected)) + return false; + + if (ParserKeyword("STORAGE").ignore(pos, expected)) + { + if (!ParserKeyword("DISK").ignore(pos, expected)) + { + if (!ParserKeyword("MEMORY").ignore(pos, expected)) + return false; + } + } + + return true; + } +}; + +bool ParserDeclareTableOptions::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + return ParserDeclareOption{ + { + OptionDescribe("AUTO_INCREMENT", "auto_increment", std::make_shared()), + OptionDescribe("AVG_ROW_LENGTH", "avg_row_length", std::make_shared()), + OptionDescribe("CHARACTER SET", "character_set", std::make_shared()), + OptionDescribe("DEFAULT CHARACTER SET", "character_set", std::make_shared()), + OptionDescribe("CHECKSUM", "checksum", std::make_shared>()), + OptionDescribe("COLLATE", "collate", std::make_shared()), + OptionDescribe("DEFAULT COLLATE", "collate", std::make_shared()), + OptionDescribe("COMMENT", "comment", std::make_shared()), + OptionDescribe("COMPRESSION", "compression", std::make_shared()), + OptionDescribe("CONNECTION", "connection", std::make_shared()), + OptionDescribe("DATA DIRECTORY", "data_directory", std::make_shared()), + OptionDescribe("INDEX DIRECTORY", "index_directory", std::make_shared()), + OptionDescribe("DELAY_KEY_WRITE", "delay_key_write", std::make_shared>()), + OptionDescribe("ENCRYPTION", "encryption", std::make_shared()), + OptionDescribe("ENGINE", "engine", std::make_shared()), + OptionDescribe("INSERT_METHOD", "insert_method", std::make_shared()), + OptionDescribe("KEY_BLOCK_SIZE", "key_block_size", std::make_shared()), + OptionDescribe("MAX_ROWS", "max_rows", std::make_shared()), + OptionDescribe("MIN_ROWS", "min_rows", std::make_shared()), + OptionDescribe("PACK_KEYS", "pack_keys", std::make_shared>()), + OptionDescribe("PASSWORD", "password", std::make_shared()), + OptionDescribe("ROW_FORMAT", "row_format", std::make_shared()), + OptionDescribe("STATS_AUTO_RECALC", "stats_auto_recalc", std::make_shared>()), + OptionDescribe("STATS_PERSISTENT", "stats_persistent", std::make_shared>()), + OptionDescribe("STATS_SAMPLE_PAGES", "stats_sample_pages", std::make_shared()), + OptionDescribe("TABLESPACE", "tablespace", std::make_shared()), + OptionDescribe("UNION", "union", std::make_shared()), + } + }.parse(pos, node, expected); +} + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareTableOptions.h b/src/Parsers/MySQL/ASTDeclareTableOptions.h new file mode 100644 index 00000000000..38697eae754 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareTableOptions.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ParserDeclareTableOptions : public IParserBase +{ +protected: + const char * getName() const override { return "table options declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} + +} diff --git a/src/Parsers/MySQL/tests/gtest_column_parser.cpp b/src/Parsers/MySQL/tests/gtest_column_parser.cpp new file mode 100644 index 00000000000..ef6371f71d9 --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_column_parser.cpp @@ -0,0 +1,60 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserColumn, AllNonGeneratedColumnOption) +{ + ParserDeclareColumn p_column; + + String input = "col_01 VARCHAR(100) NOT NULL DEFAULT NULL AUTO_INCREMENT UNIQUE KEY PRIMARY KEY COMMENT 'column comment' COLLATE utf-8 " + "COLUMN_FORMAT FIXED STORAGE MEMORY REFERENCES tbl_name (col_01) CHECK 1"; + ASTPtr ast = parseQuery(p_column, input.data(), input.data() + input.size(), "", 0, 0); + EXPECT_EQ(ast->as()->name, "col_01"); + EXPECT_EQ(ast->as()->data_type->as()->name, "VARCHAR"); + EXPECT_EQ(ast->as()->data_type->as()->arguments->children[0]->as()->value.safeGet(), 100); + + ASTDeclareOptions * declare_options = ast->as()->column_options->as(); + EXPECT_EQ(declare_options->changes["is_null"]->as()->value.safeGet(), 0); + EXPECT_TRUE(declare_options->changes["default"]->as()->value.isNull()); + EXPECT_EQ(declare_options->changes["auto_increment"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["unique_key"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["primary_key"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["comment"]->as()->value.safeGet(), "column comment"); + EXPECT_EQ(declare_options->changes["collate"]->as()->name, "utf-8"); + EXPECT_EQ(declare_options->changes["column_format"]->as()->name, "FIXED"); + EXPECT_EQ(declare_options->changes["storage"]->as()->name, "MEMORY"); + EXPECT_TRUE(declare_options->changes["reference"]->as()); + EXPECT_TRUE(declare_options->changes["constraint"]->as()); +} + +TEST(ParserColumn, AllGeneratedColumnOption) +{ + ParserDeclareColumn p_column; + + String input = "col_01 VARCHAR(100) NULL UNIQUE KEY PRIMARY KEY COMMENT 'column comment' COLLATE utf-8 " + "REFERENCES tbl_name (col_01) CHECK 1 GENERATED ALWAYS AS (1) STORED"; + ASTPtr ast = parseQuery(p_column, input.data(), input.data() + input.size(), "", 0, 0); + EXPECT_EQ(ast->as()->name, "col_01"); + EXPECT_EQ(ast->as()->data_type->as()->name, "VARCHAR"); + EXPECT_EQ(ast->as()->data_type->as()->arguments->children[0]->as()->value.safeGet(), 100); + + ASTDeclareOptions * declare_options = ast->as()->column_options->as(); + EXPECT_EQ(declare_options->changes["is_null"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["unique_key"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["primary_key"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["comment"]->as()->value.safeGet(), "column comment"); + EXPECT_EQ(declare_options->changes["collate"]->as()->name, "utf-8"); + EXPECT_EQ(declare_options->changes["generated"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["is_stored"]->as()->value.safeGet(), 1); + EXPECT_TRUE(declare_options->changes["reference"]->as()); + EXPECT_TRUE(declare_options->changes["constraint"]->as()); +} diff --git a/src/Parsers/MySQL/tests/gtest_constraint_parser.cpp b/src/Parsers/MySQL/tests/gtest_constraint_parser.cpp new file mode 100644 index 00000000000..7f0e9a8060d --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_constraint_parser.cpp @@ -0,0 +1,57 @@ +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserConstraint, CheckConstraint) +{ + /// [CONSTRAINT [symbol]] CHECK (expr) [[NOT] ENFORCED] + ParserDeclareConstraint p_constraint; + + String constraint_01 = "CONSTRAINT symbol_name CHECK col_01 = 1"; + ASTPtr ast_constraint_01 = parseQuery(p_constraint, constraint_01.data(), constraint_01.data() + constraint_01.size(), "", 0, 0); + EXPECT_EQ(ast_constraint_01->as()->constraint_name, "symbol_name"); + auto check_expression_01 = ast_constraint_01->as()->check_expression->as(); + EXPECT_EQ(check_expression_01->name, "equals"); + EXPECT_EQ(check_expression_01->arguments->children[0]->as()->name, "col_01"); + EXPECT_EQ(check_expression_01->arguments->children[1]->as()->value.safeGet(), 1); + + String constraint_02 = "CONSTRAINT CHECK col_01 = 1"; + ASTPtr ast_constraint_02 = parseQuery(p_constraint, constraint_02.data(), constraint_02.data() + constraint_02.size(), "", 0, 0); + EXPECT_EQ(ast_constraint_02->as()->constraint_name, ""); + auto check_expression_02 = ast_constraint_02->as()->check_expression->as(); + EXPECT_EQ(check_expression_02->name, "equals"); + EXPECT_EQ(check_expression_02->arguments->children[0]->as()->name, "col_01"); + EXPECT_EQ(check_expression_02->arguments->children[1]->as()->value.safeGet(), 1); + + String constraint_03 = "CHECK col_01 = 1"; + ASTPtr ast_constraint_03 = parseQuery(p_constraint, constraint_03.data(), constraint_03.data() + constraint_03.size(), "", 0, 0); + EXPECT_EQ(ast_constraint_03->as()->constraint_name, ""); + auto check_expression_03 = ast_constraint_03->as()->check_expression->as(); + EXPECT_EQ(check_expression_03->name, "equals"); + EXPECT_EQ(check_expression_03->arguments->children[0]->as()->name, "col_01"); + EXPECT_EQ(check_expression_03->arguments->children[1]->as()->value.safeGet(), 1); + + String constraint_04 = "CONSTRAINT CHECK col_01 = 1 ENFORCED"; + ASTPtr ast_constraint_04 = parseQuery(p_constraint, constraint_04.data(), constraint_04.data() + constraint_04.size(), "", 0, 0); + EXPECT_TRUE(ast_constraint_04->as()->enforced); + EXPECT_EQ(ast_constraint_04->as()->constraint_name, ""); + auto check_expression_04 = ast_constraint_04->as()->check_expression->as(); + EXPECT_EQ(check_expression_04->name, "equals"); + EXPECT_EQ(check_expression_04->arguments->children[0]->as()->name, "col_01"); + EXPECT_EQ(check_expression_04->arguments->children[1]->as()->value.safeGet(), 1); + + String constraint_05 = "CONSTRAINT CHECK col_01 = 1 NOT ENFORCED"; + ASTPtr ast_constraint_05 = parseQuery(p_constraint, constraint_05.data(), constraint_05.data() + constraint_05.size(), "", 0, 0); + EXPECT_FALSE(ast_constraint_05->as()->enforced); + EXPECT_EQ(ast_constraint_05->as()->constraint_name, ""); + auto check_expression_05 = ast_constraint_05->as()->check_expression->as(); + EXPECT_EQ(check_expression_05->name, "equals"); + EXPECT_EQ(check_expression_05->arguments->children[0]->as()->name, "col_01"); + EXPECT_EQ(check_expression_05->arguments->children[1]->as()->value.safeGet(), 1); +} diff --git a/src/Parsers/MySQL/tests/gtest_create_parser.cpp b/src/Parsers/MySQL/tests/gtest_create_parser.cpp new file mode 100644 index 00000000000..5f752c29a7d --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_create_parser.cpp @@ -0,0 +1,33 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(CreateTableParser, LikeCreate) +{ + ParserCreateQuery p_create_query; + String like_create_01 = "CREATE TABLE IF NOT EXISTS table_name LIKE table_name_01"; + parseQuery(p_create_query, like_create_01.data(), like_create_01.data() + like_create_01.size(), "", 0, 0); + String like_create_02 = "CREATE TABLE IF NOT EXISTS table_name (LIKE table_name_01)"; + parseQuery(p_create_query, like_create_02.data(), like_create_02.data() + like_create_02.size(), "", 0, 0); +} + +TEST(CreateTableParser, SimpleCreate) +{ + ParserCreateQuery p_create_query; + String input = "CREATE TABLE IF NOT EXISTS table_name(col_01 VARCHAR(100), INDEX (col_01), CHECK 1) ENGINE INNODB PARTITION BY HASH(col_01)"; + ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); + EXPECT_TRUE(ast->as()->if_not_exists); + EXPECT_EQ(ast->as()->columns_list->as()->columns->children.size(), 1); + EXPECT_EQ(ast->as()->columns_list->as()->indices->children.size(), 1); + EXPECT_EQ(ast->as()->columns_list->as()->constraints->children.size(), 1); + EXPECT_EQ(ast->as()->table_options->as()->changes["engine"]->as()->name, "INNODB"); + EXPECT_TRUE(ast->as()->partition_options->as()); +} diff --git a/src/Parsers/MySQL/tests/gtest_index_parser.cpp b/src/Parsers/MySQL/tests/gtest_index_parser.cpp new file mode 100644 index 00000000000..02b3b10acff --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_index_parser.cpp @@ -0,0 +1,125 @@ +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserIndex, AllIndexOptions) +{ + String input = "INDEX (col_01, col_02(100), col_03 DESC) KEY_BLOCK_SIZE 3 USING HASH WITH PARSER parser_name COMMENT 'index comment' VISIBLE"; + + ParserDeclareIndex p_index; + ASTPtr ast = parseQuery(p_index, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclareIndex * declare_index = ast->as(); + EXPECT_EQ(declare_index->index_columns->children[0]->as()->name, "col_01"); + EXPECT_EQ(declare_index->index_columns->children[1]->as()->name, "col_02"); + EXPECT_EQ(declare_index->index_columns->children[1]->as()->arguments->children[0]->as()->value.safeGet(), 100); + EXPECT_EQ(declare_index->index_columns->children[2]->as()->name, "col_03"); + ASTDeclareOptions * declare_options = declare_index->index_options->as(); + EXPECT_EQ(declare_options->changes["key_block_size"]->as()->value.safeGet(), 3); + EXPECT_EQ(declare_options->changes["index_type"]->as()->name, "HASH"); + EXPECT_EQ(declare_options->changes["comment"]->as()->value.safeGet(), "index comment"); + EXPECT_EQ(declare_options->changes["visible"]->as()->value.safeGet(), 1); +} + +TEST(ParserIndex, OptionalIndexOptions) +{ + String input = "INDEX (col_01, col_02(100), col_03 DESC) USING HASH INVISIBLE KEY_BLOCK_SIZE 3"; + + ParserDeclareIndex p_index; + ASTPtr ast = parseQuery(p_index, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclareIndex * declare_index = ast->as(); + EXPECT_EQ(declare_index->index_columns->children[0]->as()->name, "col_01"); + EXPECT_EQ(declare_index->index_columns->children[1]->as()->name, "col_02"); + EXPECT_EQ(declare_index->index_columns->children[1]->as()->arguments->children[0]->as()->value.safeGet(), 100); + EXPECT_EQ(declare_index->index_columns->children[2]->as()->name, "col_03"); + ASTDeclareOptions * declare_options = declare_index->index_options->as(); + EXPECT_EQ(declare_options->changes["index_type"]->as()->name, "HASH"); + EXPECT_EQ(declare_options->changes["visible"]->as()->value.safeGet(), 0); + EXPECT_EQ(declare_options->changes["key_block_size"]->as()->value.safeGet(), 3); +} + +TEST(ParserIndex, OrdinaryIndex) +{ + ParserDeclareIndex p_index; + String non_unique_index_01 = "KEY index_name USING HASH (col_01) INVISIBLE"; + parseQuery(p_index, non_unique_index_01.data(), non_unique_index_01.data() + non_unique_index_01.size(), "", 0, 0); + + String non_unique_index_02 = "INDEX index_name USING HASH (col_01) INVISIBLE"; + parseQuery(p_index, non_unique_index_02.data(), non_unique_index_02.data() + non_unique_index_02.size(), "", 0, 0); + + String fulltext_index_01 = "FULLTEXT index_name (col_01) INVISIBLE"; + parseQuery(p_index, fulltext_index_01.data(), fulltext_index_01.data() + fulltext_index_01.size(), "", 0, 0); + + String fulltext_index_02 = "FULLTEXT INDEX index_name (col_01) INVISIBLE"; + parseQuery(p_index, fulltext_index_02.data(), fulltext_index_02.data() + fulltext_index_02.size(), "", 0, 0); + + String fulltext_index_03 = "FULLTEXT KEY index_name (col_01) INVISIBLE"; + parseQuery(p_index, fulltext_index_03.data(), fulltext_index_03.data() + fulltext_index_03.size(), "", 0, 0); + + String spatial_index_01 = "SPATIAL index_name (col_01) INVISIBLE"; + parseQuery(p_index, spatial_index_01.data(), spatial_index_01.data() + spatial_index_01.size(), "", 0, 0); + + String spatial_index_02 = "SPATIAL INDEX index_name (col_01) INVISIBLE"; + parseQuery(p_index, spatial_index_02.data(), spatial_index_02.data() + spatial_index_02.size(), "", 0, 0); + + String spatial_index_03 = "SPATIAL KEY index_name (col_01) INVISIBLE"; + parseQuery(p_index, spatial_index_03.data(), spatial_index_03.data() + spatial_index_03.size(), "", 0, 0); +} + +TEST(ParserIndex, ConstraintIndex) +{ + ParserDeclareIndex p_index; + + String primary_key_01 = "PRIMARY KEY (col_01) INVISIBLE"; + parseQuery(p_index, primary_key_01.data(), primary_key_01.data() + primary_key_01.size(), "", 0, 0); + + String primary_key_02 = "PRIMARY KEY USING BTREE (col_01) INVISIBLE"; + parseQuery(p_index, primary_key_02.data(), primary_key_02.data() + primary_key_02.size(), "", 0, 0); + + String primary_key_03 = "CONSTRAINT PRIMARY KEY USING BTREE (col_01) INVISIBLE"; + parseQuery(p_index, primary_key_03.data(), primary_key_03.data() + primary_key_03.size(), "", 0, 0); + + String primary_key_04 = "CONSTRAINT index_name PRIMARY KEY USING BTREE (col_01) INVISIBLE"; + parseQuery(p_index, primary_key_04.data(), primary_key_04.data() + primary_key_04.size(), "", 0, 0); + + String unique_key_01 = "UNIQUE (col_01) INVISIBLE"; + parseQuery(p_index, unique_key_01.data(), unique_key_01.data() + unique_key_01.size(), "", 0, 0); + + String unique_key_02 = "UNIQUE INDEX (col_01) INVISIBLE"; + parseQuery(p_index, unique_key_02.data(), unique_key_02.data() + unique_key_02.size(), "", 0, 0); + + String unique_key_03 = "UNIQUE KEY (col_01) INVISIBLE"; + parseQuery(p_index, unique_key_03.data(), unique_key_03.data() + unique_key_03.size(), "", 0, 0); + + String unique_key_04 = "UNIQUE KEY index_name (col_01) INVISIBLE"; + parseQuery(p_index, unique_key_04.data(), unique_key_04.data() + unique_key_04.size(), "", 0, 0); + + String unique_key_05 = "UNIQUE KEY index_name USING HASH (col_01) INVISIBLE"; + parseQuery(p_index, unique_key_05.data(), unique_key_05.data() + unique_key_05.size(), "", 0, 0); + + String unique_key_06 = "CONSTRAINT UNIQUE KEY index_name USING HASH (col_01) INVISIBLE"; + parseQuery(p_index, unique_key_06.data(), unique_key_06.data() + unique_key_06.size(), "", 0, 0); + + String unique_key_07 = "CONSTRAINT index_name UNIQUE KEY index_name_1 USING HASH (col_01) INVISIBLE"; + parseQuery(p_index, unique_key_07.data(), unique_key_07.data() + unique_key_07.size(), "", 0, 0); + + String foreign_key_01 = "FOREIGN KEY (col_01) REFERENCES tbl_name (col_01)"; + parseQuery(p_index, foreign_key_01.data(), foreign_key_01.data() + foreign_key_01.size(), "", 0, 0); + + String foreign_key_02 = "FOREIGN KEY index_name (col_01) REFERENCES tbl_name (col_01)"; + parseQuery(p_index, foreign_key_02.data(), foreign_key_02.data() + foreign_key_02.size(), "", 0, 0); + + String foreign_key_03 = "CONSTRAINT FOREIGN KEY index_name (col_01) REFERENCES tbl_name (col_01)"; + parseQuery(p_index, foreign_key_03.data(), foreign_key_03.data() + foreign_key_03.size(), "", 0, 0); + + String foreign_key_04 = "CONSTRAINT index_name FOREIGN KEY index_name_01 (col_01) REFERENCES tbl_name (col_01)"; + parseQuery(p_index, foreign_key_04.data(), foreign_key_04.data() + foreign_key_04.size(), "", 0, 0); +} diff --git a/src/Parsers/MySQL/tests/gtest_partition_options_parser.cpp b/src/Parsers/MySQL/tests/gtest_partition_options_parser.cpp new file mode 100644 index 00000000000..1651efcb966 --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_partition_options_parser.cpp @@ -0,0 +1,167 @@ +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserPartitionOptions, HashPatitionOptions) +{ + String hash_partition = "PARTITION BY HASH(col_01)"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast_01 = parseQuery(p_partition_options, hash_partition.data(), hash_partition.data() + hash_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_01 = ast_01->as(); + EXPECT_EQ(declare_partition_options_01->partition_type, "hash"); + EXPECT_EQ(declare_partition_options_01->partition_expression->as()->name, "col_01"); + + String linear_hash_partition = "PARTITION BY LINEAR HASH(col_01)"; + ASTPtr ast_02 = parseQuery(p_partition_options, linear_hash_partition.data(), linear_hash_partition.data() + linear_hash_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_02 = ast_02->as(); + EXPECT_EQ(declare_partition_options_02->partition_type, "linear_hash"); + EXPECT_EQ(declare_partition_options_02->partition_expression->as()->name, "col_01"); +} + +TEST(ParserPartitionOptions, KeyPatitionOptions) +{ + String key_partition = "PARTITION BY KEY(col_01)"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast_01 = parseQuery(p_partition_options, key_partition.data(), key_partition.data() + key_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_01 = ast_01->as(); + EXPECT_EQ(declare_partition_options_01->partition_type, "key"); + EXPECT_EQ(declare_partition_options_01->partition_expression->as()->name, "col_01"); + + String linear_key_partition = "PARTITION BY LINEAR KEY(col_01, col_02)"; + ASTPtr ast_02 = parseQuery(p_partition_options, linear_key_partition.data(), linear_key_partition.data() + linear_key_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_02 = ast_02->as(); + EXPECT_EQ(declare_partition_options_02->partition_type, "linear_key"); + ASTPtr columns_list = declare_partition_options_02->partition_expression->as()->arguments; + EXPECT_EQ(columns_list->children[0]->as()->name, "col_01"); + EXPECT_EQ(columns_list->children[1]->as()->name, "col_02"); + + String key_partition_with_algorithm = "PARTITION BY KEY ALGORITHM=1 (col_01)"; + ASTPtr ast_03 = parseQuery(p_partition_options, key_partition_with_algorithm.data(), key_partition_with_algorithm.data() + key_partition_with_algorithm.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_03 = ast_03->as(); + EXPECT_EQ(declare_partition_options_03->partition_type, "key_1"); + EXPECT_EQ(declare_partition_options_03->partition_expression->as()->name, "col_01"); +} + +TEST(ParserPartitionOptions, RangePatitionOptions) +{ + String range_partition = "PARTITION BY RANGE(col_01)"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast_01 = parseQuery(p_partition_options, range_partition.data(), range_partition.data() + range_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_01 = ast_01->as(); + EXPECT_EQ(declare_partition_options_01->partition_type, "range"); + EXPECT_EQ(declare_partition_options_01->partition_expression->as()->name, "col_01"); + + String range_columns_partition = "PARTITION BY RANGE COLUMNS(col_01, col_02)"; + ASTPtr ast_02 = parseQuery(p_partition_options, range_columns_partition.data(), range_columns_partition.data() + range_columns_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_02 = ast_02->as(); + EXPECT_EQ(declare_partition_options_02->partition_type, "range"); + ASTPtr columns_list = declare_partition_options_02->partition_expression->as()->arguments; + EXPECT_EQ(columns_list->children[0]->as()->name, "col_01"); + EXPECT_EQ(columns_list->children[1]->as()->name, "col_02"); +} + +TEST(ParserPartitionOptions, ListPatitionOptions) +{ + String range_partition = "PARTITION BY LIST(col_01)"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast_01 = parseQuery(p_partition_options, range_partition.data(), range_partition.data() + range_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_01 = ast_01->as(); + EXPECT_EQ(declare_partition_options_01->partition_type, "list"); + EXPECT_EQ(declare_partition_options_01->partition_expression->as()->name, "col_01"); + + String range_columns_partition = "PARTITION BY LIST COLUMNS(col_01, col_02)"; + ASTPtr ast_02 = parseQuery(p_partition_options, range_columns_partition.data(), range_columns_partition.data() + range_columns_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_02 = ast_02->as(); + EXPECT_EQ(declare_partition_options_02->partition_type, "list"); + ASTPtr columns_list = declare_partition_options_02->partition_expression->as()->arguments; + EXPECT_EQ(columns_list->children[0]->as()->name, "col_01"); + EXPECT_EQ(columns_list->children[1]->as()->name, "col_02"); +} + +TEST(ParserPartitionOptions, PatitionNumberOptions) +{ + String numbers_partition = "PARTITION BY KEY(col_01) PARTITIONS 2"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast = parseQuery(p_partition_options, numbers_partition.data(), numbers_partition.data() + numbers_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options = ast->as(); + EXPECT_EQ(declare_partition_options->partition_type, "key"); + EXPECT_EQ(declare_partition_options->partition_expression->as()->name, "col_01"); + EXPECT_EQ(declare_partition_options->partition_numbers->as()->value.safeGet(), 2); +} + +TEST(ParserPartitionOptions, PatitionWithSubpartitionOptions) +{ + String partition_with_subpartition = "PARTITION BY KEY(col_01) PARTITIONS 3 SUBPARTITION BY HASH(col_02) SUBPARTITIONS 4"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast = parseQuery(p_partition_options, partition_with_subpartition.data(), partition_with_subpartition.data() + partition_with_subpartition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options = ast->as(); + EXPECT_EQ(declare_partition_options->partition_type, "key"); + EXPECT_EQ(declare_partition_options->partition_expression->as()->name, "col_01"); + EXPECT_EQ(declare_partition_options->partition_numbers->as()->value.safeGet(), 3); + EXPECT_EQ(declare_partition_options->subpartition_type, "hash"); + EXPECT_EQ(declare_partition_options->subpartition_expression->as()->name, "col_02"); + EXPECT_EQ(declare_partition_options->subpartition_numbers->as()->value.safeGet(), 4); +} + +TEST(ParserPartitionOptions, PatitionOptionsWithDeclarePartition) +{ + String partition_options_with_declare = "PARTITION BY KEY(col_01) PARTITIONS 3 SUBPARTITION BY HASH(col_02) SUBPARTITIONS 4 (PARTITION partition_name)"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast = parseQuery(p_partition_options, + partition_options_with_declare.data(), + partition_options_with_declare.data() + partition_options_with_declare.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options = ast->as(); + EXPECT_EQ(declare_partition_options->partition_type, "key"); + EXPECT_EQ(declare_partition_options->partition_expression->as()->name, "col_01"); + EXPECT_EQ(declare_partition_options->partition_numbers->as()->value.safeGet(), 3); + EXPECT_EQ(declare_partition_options->subpartition_type, "hash"); + EXPECT_EQ(declare_partition_options->subpartition_expression->as()->name, "col_02"); + EXPECT_EQ(declare_partition_options->subpartition_numbers->as()->value.safeGet(), 4); + EXPECT_TRUE(declare_partition_options->declare_partitions->as()->children[0]->as()); +} + +TEST(ParserPartitionOptions, PatitionOptionsWithDeclarePartitions) +{ + String partition_options_with_declare = "PARTITION BY KEY(col_01) PARTITIONS 3 SUBPARTITION BY HASH(col_02) SUBPARTITIONS 4 (PARTITION partition_01, PARTITION partition_02)"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast = parseQuery(p_partition_options, + partition_options_with_declare.data(), + partition_options_with_declare.data() + partition_options_with_declare.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options = ast->as(); + EXPECT_EQ(declare_partition_options->partition_type, "key"); + EXPECT_EQ(declare_partition_options->partition_expression->as()->name, "col_01"); + EXPECT_EQ(declare_partition_options->partition_numbers->as()->value.safeGet(), 3); + EXPECT_EQ(declare_partition_options->subpartition_type, "hash"); + EXPECT_EQ(declare_partition_options->subpartition_expression->as()->name, "col_02"); + EXPECT_EQ(declare_partition_options->subpartition_numbers->as()->value.safeGet(), 4); + EXPECT_TRUE(declare_partition_options->declare_partitions->as()->children[0]->as()); + EXPECT_TRUE(declare_partition_options->declare_partitions->as()->children[1]->as()); +} diff --git a/src/Parsers/MySQL/tests/gtest_partition_parser.cpp b/src/Parsers/MySQL/tests/gtest_partition_parser.cpp new file mode 100644 index 00000000000..48e8a9f53c6 --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_partition_parser.cpp @@ -0,0 +1,149 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserPartition, AllPatitionOptions) +{ + String input = "PARTITION partition_name ENGINE = engine_name COMMENT 'partition comment'" + " INDEX DIRECTORY 'index_directory' DATA DIRECTORY 'data_directory' max_rows 1000 MIN_ROWs 0" + " TABLESPACE table_space_name"; + + ParserDeclarePartition p_partition; + ASTPtr ast = parseQuery(p_partition, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition = ast->as(); + EXPECT_EQ(declare_partition->partition_name, "partition_name"); + ASTDeclareOptions * declare_options = declare_partition->options->as(); + EXPECT_EQ(declare_options->changes["engine"]->as()->name, "engine_name"); + EXPECT_EQ(declare_options->changes["comment"]->as()->value.safeGet(), "partition comment"); + EXPECT_EQ(declare_options->changes["data_directory"]->as()->value.safeGet(), "data_directory"); + EXPECT_EQ(declare_options->changes["index_directory"]->as()->value.safeGet(), "index_directory"); + EXPECT_EQ(declare_options->changes["min_rows"]->as()->value.safeGet(), 0); + EXPECT_EQ(declare_options->changes["max_rows"]->as()->value.safeGet(), 1000); + EXPECT_EQ(declare_options->changes["tablespace"]->as()->name, "table_space_name"); +} + +TEST(ParserPartition, OptionalPatitionOptions) +{ + String input = "PARTITION partition_name STORAGE engine = engine_name max_rows 1000 min_rows 0 tablespace table_space_name"; + ParserDeclarePartition p_partition; + ASTPtr ast = parseQuery(p_partition, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition = ast->as(); + EXPECT_EQ(declare_partition->partition_name, "partition_name"); + ASTDeclareOptions * declare_options = declare_partition->options->as(); + EXPECT_EQ(declare_options->changes["engine"]->as()->name, "engine_name"); + EXPECT_EQ(declare_options->changes["min_rows"]->as()->value.safeGet(), 0); + EXPECT_EQ(declare_options->changes["max_rows"]->as()->value.safeGet(), 1000); + EXPECT_EQ(declare_options->changes["tablespace"]->as()->name, "table_space_name"); +} + +TEST(ParserPartition, PatitionOptionsWithLessThan) +{ + ParserDeclarePartition p_partition; + String partition_01 = "PARTITION partition_01 VALUES LESS THAN (1991) STORAGE engine = engine_name"; + ASTPtr ast_partition_01 = parseQuery(p_partition, partition_01.data(), partition_01.data() + partition_01.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_01 = ast_partition_01->as(); + EXPECT_EQ(declare_partition_01->partition_name, "partition_01"); + EXPECT_EQ(declare_partition_01->less_than->as()->value.safeGet(), 1991); + ASTDeclareOptions * declare_options_01 = declare_partition_01->options->as(); + EXPECT_EQ(declare_options_01->changes["engine"]->as()->name, "engine_name"); + + String partition_02 = "PARTITION partition_02 VALUES LESS THAN MAXVALUE STORAGE engine = engine_name"; + ASTPtr ast_partition_02 = parseQuery(p_partition, partition_02.data(), partition_02.data() + partition_02.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_02 = ast_partition_02->as(); + EXPECT_EQ(declare_partition_02->partition_name, "partition_02"); + EXPECT_EQ(declare_partition_02->less_than->as()->name, "MAXVALUE"); + ASTDeclareOptions * declare_options_02 = declare_partition_02->options->as(); + EXPECT_EQ(declare_options_02->changes["engine"]->as()->name, "engine_name"); + + String partition_03 = "PARTITION partition_03 VALUES LESS THAN (50, MAXVALUE) STORAGE engine = engine_name"; + ASTPtr ast_partition_03 = parseQuery(p_partition, partition_03.data(), partition_03.data() + partition_03.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_03 = ast_partition_03->as(); + EXPECT_EQ(declare_partition_03->partition_name, "partition_03"); + ASTPtr declare_partition_03_argument = declare_partition_03->less_than->as()->arguments; + EXPECT_EQ(declare_partition_03_argument->children[0]->as()->value.safeGet(), 50); + EXPECT_EQ(declare_partition_03_argument->children[1]->as()->name, "MAXVALUE"); + ASTDeclareOptions * declare_options_03 = declare_partition_03->options->as(); + EXPECT_EQ(declare_options_03->changes["engine"]->as()->name, "engine_name"); + + String partition_04 = "PARTITION partition_04 VALUES LESS THAN (MAXVALUE, MAXVALUE) STORAGE engine = engine_name"; + ASTPtr ast_partition_04 = parseQuery(p_partition, partition_04.data(), partition_04.data() + partition_04.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_04 = ast_partition_04->as(); + EXPECT_EQ(declare_partition_04->partition_name, "partition_04"); + ASTPtr declare_partition_04_argument = declare_partition_04->less_than->as()->arguments; + EXPECT_EQ(declare_partition_04_argument->children[0]->as()->name, "MAXVALUE"); + EXPECT_EQ(declare_partition_04_argument->children[1]->as()->name, "MAXVALUE"); + ASTDeclareOptions * declare_options_04 = declare_partition_04->options->as(); + EXPECT_EQ(declare_options_04->changes["engine"]->as()->name, "engine_name"); +} + +TEST(ParserPartition, PatitionOptionsWithInExpression) +{ + ParserDeclarePartition p_partition; + String partition_01 = "PARTITION partition_01 VALUES IN (NULL, 1991, MAXVALUE) STORAGE engine = engine_name"; + ASTPtr ast_partition_01 = parseQuery(p_partition, partition_01.data(), partition_01.data() + partition_01.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_01 = ast_partition_01->as(); + EXPECT_EQ(declare_partition_01->partition_name, "partition_01"); + ASTPtr declare_partition_01_argument = declare_partition_01->in_expression->as()->arguments; + EXPECT_TRUE(declare_partition_01_argument->children[0]->as()->value.isNull()); + EXPECT_EQ(declare_partition_01_argument->children[1]->as()->value.safeGet(), 1991); + EXPECT_EQ(declare_partition_01_argument->children[2]->as()->name, "MAXVALUE"); + ASTDeclareOptions * declare_options_01 = declare_partition_01->options->as(); + EXPECT_EQ(declare_options_01->changes["engine"]->as()->name, "engine_name"); + + String partition_02 = "PARTITION partition_02 VALUES IN ((NULL, 1991), (1991, NULL), (MAXVALUE, MAXVALUE)) STORAGE engine = engine_name"; + ASTPtr ast_partition_02 = parseQuery(p_partition, partition_02.data(), partition_02.data() + partition_02.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_02 = ast_partition_02->as(); + EXPECT_EQ(declare_partition_02->partition_name, "partition_02"); + ASTPtr declare_partition_02_argument = declare_partition_02->in_expression->as()->arguments; + + ASTPtr argument_01 = declare_partition_02_argument->children[0]; + EXPECT_TRUE(argument_01->as()->value.safeGet()[0].isNull()); + EXPECT_EQ(argument_01->as()->value.safeGet()[1].safeGet(), 1991); + + ASTPtr argument_02 = declare_partition_02_argument->children[1]; + EXPECT_EQ(argument_02->as()->value.safeGet()[0].safeGet(), 1991); + EXPECT_TRUE(argument_02->as()->value.safeGet()[1].isNull()); + + ASTPtr argument_03 = declare_partition_02_argument->children[2]->as()->arguments; + EXPECT_EQ(argument_03->as()->children[0]->as()->name, "MAXVALUE"); + EXPECT_EQ(argument_03->as()->children[1]->as()->name, "MAXVALUE"); + + ASTDeclareOptions * declare_options_02 = declare_partition_02->options->as(); + EXPECT_EQ(declare_options_02->changes["engine"]->as()->name, "engine_name"); +} + +TEST(ParserPartition, PatitionOptionsWithSubpartitions) +{ + ParserDeclarePartition p_partition; + String partition_01 = "PARTITION partition_01 VALUES IN (NULL, 1991, MAXVALUE) STORAGE engine = engine_name (SUBPARTITION s_p01)"; + ASTPtr ast_partition_01 = parseQuery(p_partition, partition_01.data(), partition_01.data() + partition_01.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_01 = ast_partition_01->as(); + EXPECT_EQ(declare_partition_01->partition_name, "partition_01"); + EXPECT_TRUE(declare_partition_01->subpartitions->as()->children[0]->as()); + + String partition_02 = "PARTITION partition_02 VALUES IN (NULL, 1991, MAXVALUE) STORAGE engine = engine_name (SUBPARTITION s_p01, SUBPARTITION s_p02)"; + ASTPtr ast_partition_02 = parseQuery(p_partition, partition_02.data(), partition_02.data() + partition_02.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_02 = ast_partition_02->as(); + EXPECT_EQ(declare_partition_02->partition_name, "partition_02"); + EXPECT_TRUE(declare_partition_02->subpartitions->as()->children[0]->as()); + EXPECT_TRUE(declare_partition_02->subpartitions->as()->children[1]->as()); +} + diff --git a/src/Parsers/MySQL/tests/gtest_reference_parser.cpp b/src/Parsers/MySQL/tests/gtest_reference_parser.cpp new file mode 100644 index 00000000000..694558b9cc3 --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_reference_parser.cpp @@ -0,0 +1,92 @@ +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserReference, SimpleReference) +{ + ParserDeclareReference p_reference; + + String reference_01 = "REFERENCES table_name (ref_col_01)"; + ASTPtr ast_reference_01 = parseQuery(p_reference, reference_01.data(), reference_01.data() + reference_01.size(), "", 0, 0); + EXPECT_EQ(ast_reference_01->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_01->as()->reference_expression->as()->name, "ref_col_01"); + + String reference_02 = "REFERENCES table_name (ref_col_01, ref_col_02)"; + ASTPtr ast_reference_02 = parseQuery(p_reference, reference_02.data(), reference_02.data() + reference_02.size(), "", 0, 0); + EXPECT_EQ(ast_reference_02->as()->reference_table_name, "table_name"); + ASTPtr arguments = ast_reference_02->as()->reference_expression->as()->arguments; + EXPECT_EQ(arguments->children[0]->as()->name, "ref_col_01"); + EXPECT_EQ(arguments->children[1]->as()->name, "ref_col_02"); +} + +TEST(ParserReference, ReferenceDifferenceKind) +{ + ParserDeclareReference p_reference; + String reference_01 = "REFERENCES table_name (ref_col_01) MATCH FULL"; + ASTPtr ast_reference_01 = parseQuery(p_reference, reference_01.data(), reference_01.data() + reference_01.size(), "", 0, 0); + EXPECT_EQ(ast_reference_01->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_01->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_01->as()->kind, ASTDeclareReference::MATCH_FULL); + + String reference_02 = "REFERENCES table_name (ref_col_01) MATCH PARTIAL"; + ASTPtr ast_reference_02 = parseQuery(p_reference, reference_02.data(), reference_02.data() + reference_02.size(), "", 0, 0); + EXPECT_EQ(ast_reference_02->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_02->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_02->as()->kind, ASTDeclareReference::MATCH_PARTIAL); + + String reference_03 = "REFERENCES table_name (ref_col_01) MATCH SIMPLE"; + ASTPtr ast_reference_03 = parseQuery(p_reference, reference_03.data(), reference_03.data() + reference_03.size(), "", 0, 0); + EXPECT_EQ(ast_reference_03->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_03->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_03->as()->kind, ASTDeclareReference::MATCH_SIMPLE); +} + +TEST(ParserReference, ReferenceDifferenceOption) +{ + ParserDeclareReference p_reference; + String reference_01 = "REFERENCES table_name (ref_col_01) MATCH FULL ON DELETE RESTRICT ON UPDATE RESTRICT"; + ASTPtr ast_reference_01 = parseQuery(p_reference, reference_01.data(), reference_01.data() + reference_01.size(), "", 0, 0); + EXPECT_EQ(ast_reference_01->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_01->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_01->as()->kind, ASTDeclareReference::MATCH_FULL); + EXPECT_EQ(ast_reference_01->as()->on_delete_option, ASTDeclareReference::RESTRICT); + EXPECT_EQ(ast_reference_01->as()->on_update_option, ASTDeclareReference::RESTRICT); + + String reference_02 = "REFERENCES table_name (ref_col_01) MATCH FULL ON DELETE CASCADE ON UPDATE CASCADE"; + ASTPtr ast_reference_02 = parseQuery(p_reference, reference_02.data(), reference_02.data() + reference_02.size(), "", 0, 0); + EXPECT_EQ(ast_reference_02->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_02->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_02->as()->kind, ASTDeclareReference::MATCH_FULL); + EXPECT_EQ(ast_reference_02->as()->on_delete_option, ASTDeclareReference::CASCADE); + EXPECT_EQ(ast_reference_02->as()->on_update_option, ASTDeclareReference::CASCADE); + + String reference_03 = "REFERENCES table_name (ref_col_01) MATCH FULL ON DELETE SET NULL ON UPDATE SET NULL"; + ASTPtr ast_reference_03 = parseQuery(p_reference, reference_03.data(), reference_03.data() + reference_03.size(), "", 0, 0); + EXPECT_EQ(ast_reference_03->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_03->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_03->as()->kind, ASTDeclareReference::MATCH_FULL); + EXPECT_EQ(ast_reference_03->as()->on_delete_option, ASTDeclareReference::SET_NULL); + EXPECT_EQ(ast_reference_03->as()->on_update_option, ASTDeclareReference::SET_NULL); + + String reference_04 = "REFERENCES table_name (ref_col_01) MATCH FULL ON UPDATE NO ACTION ON DELETE NO ACTION"; + ASTPtr ast_reference_04 = parseQuery(p_reference, reference_04.data(), reference_04.data() + reference_04.size(), "", 0, 0); + EXPECT_EQ(ast_reference_04->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_04->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_04->as()->kind, ASTDeclareReference::MATCH_FULL); + EXPECT_EQ(ast_reference_04->as()->on_delete_option, ASTDeclareReference::NO_ACTION); + EXPECT_EQ(ast_reference_04->as()->on_update_option, ASTDeclareReference::NO_ACTION); + + String reference_05 = "REFERENCES table_name (ref_col_01) MATCH FULL ON UPDATE SET DEFAULT ON DELETE SET DEFAULT"; + ASTPtr ast_reference_05 = parseQuery(p_reference, reference_05.data(), reference_05.data() + reference_05.size(), "", 0, 0); + EXPECT_EQ(ast_reference_05->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_05->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_05->as()->kind, ASTDeclareReference::MATCH_FULL); + EXPECT_EQ(ast_reference_05->as()->on_delete_option, ASTDeclareReference::SET_DEFAULT); + EXPECT_EQ(ast_reference_05->as()->on_update_option, ASTDeclareReference::SET_DEFAULT); +} + diff --git a/src/Parsers/MySQL/tests/gtest_subpartition_parser.cpp b/src/Parsers/MySQL/tests/gtest_subpartition_parser.cpp new file mode 100644 index 00000000000..5c1cf3710ab --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_subpartition_parser.cpp @@ -0,0 +1,45 @@ +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserSubpartition, AllSubpatitionOptions) +{ + String input = "SUBPARTITION subpartition_name ENGINE = engine_name COMMENT 'subpartition comment'" + " DATA DIRECTORY 'data_directory' INDEX DIRECTORY 'index_directory' max_rows 1000 MIN_ROWs 0" + " TABLESPACE table_space_name"; + MySQLParser::ParserDeclareSubPartition p_subpartition; + ASTPtr ast = parseQuery(p_subpartition, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclareSubPartition * declare_subpartition = ast->as(); + EXPECT_EQ(declare_subpartition->logical_name, "subpartition_name"); + ASTDeclareOptions * declare_options = declare_subpartition->options->as(); + EXPECT_EQ(declare_options->changes["engine"]->as()->name, "engine_name"); + EXPECT_EQ(declare_options->changes["comment"]->as()->value.safeGet(), "subpartition comment"); + EXPECT_EQ(declare_options->changes["data_directory"]->as()->value.safeGet(), "data_directory"); + EXPECT_EQ(declare_options->changes["index_directory"]->as()->value.safeGet(), "index_directory"); + EXPECT_EQ(declare_options->changes["min_rows"]->as()->value.safeGet(), 0); + EXPECT_EQ(declare_options->changes["max_rows"]->as()->value.safeGet(), 1000); + EXPECT_EQ(declare_options->changes["tablespace"]->as()->name, "table_space_name"); +} + +TEST(ParserSubpartition, OptionalSubpatitionOptions) +{ + String input = "SUBPARTITION subpartition_name STORAGE engine = engine_name max_rows 1000 min_rows 0 tablespace table_space_name"; + MySQLParser::ParserDeclareSubPartition p_subpartition; + ASTPtr ast = parseQuery(p_subpartition, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclareSubPartition * declare_subpartition = ast->as(); + EXPECT_EQ(declare_subpartition->logical_name, "subpartition_name"); + ASTDeclareOptions * declare_options = declare_subpartition->options->as(); + EXPECT_EQ(declare_options->changes["engine"]->as()->name, "engine_name"); + EXPECT_EQ(declare_options->changes["min_rows"]->as()->value.safeGet(), 0); + EXPECT_EQ(declare_options->changes["max_rows"]->as()->value.safeGet(), 1000); + EXPECT_EQ(declare_options->changes["tablespace"]->as()->name, "table_space_name"); +} + diff --git a/src/Parsers/MySQL/tests/gtest_table_options_parser.cpp b/src/Parsers/MySQL/tests/gtest_table_options_parser.cpp new file mode 100644 index 00000000000..b051f6149bb --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_table_options_parser.cpp @@ -0,0 +1,64 @@ +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserTableOptions, AllSubpatitionOptions) +{ + String input = "AUTO_INCREMENt = 1 AVG_ROW_LENGTh 3 CHARACTER SET utf-8 CHECKSUM 1 COLLATE utf8_bin" + " COMMENT 'table option comment' COMPRESSION 'LZ4' CONNECTION 'connect_string' DATA DIRECTORY 'data_directory'" + " INDEX DIRECTORY 'index_directory' DELAY_KEY_WRITE 0 ENCRYPTION 'Y' ENGINE INNODB INSERT_METHOD NO KEY_BLOCK_SIZE 3" + " MAX_ROWS 1000 MIN_ROWS 0 PACK_KEYS DEFAULT PASSWORD 'password' ROW_FORMAT DYNAMIC STATS_AUTO_RECALC DEFAULT " + " STATS_PERSISTENT DEFAULT STATS_SAMPLE_PAGES 3 TABLESPACE tablespace_name STORAGE MEMORY UNION (table_01, table_02)"; + + ParserDeclareTableOptions p_table_options; + ASTPtr ast = parseQuery(p_table_options, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclareOptions * declare_options = ast->as(); + EXPECT_EQ(declare_options->changes["auto_increment"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["avg_row_length"]->as()->value.safeGet(), 3); + EXPECT_EQ(declare_options->changes["character_set"]->as()->name, "utf-8"); + EXPECT_EQ(declare_options->changes["checksum"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["collate"]->as()->name, "utf8_bin"); + EXPECT_EQ(declare_options->changes["comment"]->as()->value.safeGet(), "table option comment"); + EXPECT_EQ(declare_options->changes["compression"]->as()->value.safeGet(), "LZ4"); + EXPECT_EQ(declare_options->changes["connection"]->as()->value.safeGet(), "connect_string"); + EXPECT_EQ(declare_options->changes["data_directory"]->as()->value.safeGet(), "data_directory"); + EXPECT_EQ(declare_options->changes["index_directory"]->as()->value.safeGet(), "index_directory"); + EXPECT_EQ(declare_options->changes["delay_key_write"]->as()->value.safeGet(), 0); + EXPECT_EQ(declare_options->changes["encryption"]->as()->value.safeGet(), "Y"); + EXPECT_EQ(declare_options->changes["engine"]->as()->name, "INNODB"); + EXPECT_EQ(declare_options->changes["insert_method"]->as()->name, "NO"); + EXPECT_EQ(declare_options->changes["key_block_size"]->as()->value.safeGet(), 3); + + EXPECT_EQ(declare_options->changes["max_rows"]->as()->value.safeGet(), 1000); + EXPECT_EQ(declare_options->changes["min_rows"]->as()->value.safeGet(), 0); + EXPECT_EQ(declare_options->changes["pack_keys"]->as()->name, "DEFAULT"); + EXPECT_EQ(declare_options->changes["password"]->as()->value.safeGet(), "password"); + EXPECT_EQ(declare_options->changes["row_format"]->as()->name, "DYNAMIC"); + EXPECT_EQ(declare_options->changes["stats_auto_recalc"]->as()->name, "DEFAULT"); + EXPECT_EQ(declare_options->changes["stats_persistent"]->as()->name, "DEFAULT"); + EXPECT_EQ(declare_options->changes["stats_sample_pages"]->as()->value.safeGet(), 3); + EXPECT_EQ(declare_options->changes["tablespace"]->as()->name, "tablespace_name"); + + ASTPtr arguments = declare_options->changes["union"]->as()->arguments; + EXPECT_EQ(arguments->children[0]->as()->name, "table_01"); + EXPECT_EQ(arguments->children[1]->as()->name, "table_02"); +} + +TEST(ParserTableOptions, OptionalTableOptions) +{ + String input = "STATS_AUTO_RECALC DEFAULT AUTO_INCREMENt = 1 "; + ParserDeclareTableOptions p_table_options; + ASTPtr ast = parseQuery(p_table_options, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclareOptions * declare_options = ast->as(); + EXPECT_EQ(declare_options->changes["auto_increment"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["stats_auto_recalc"]->as()->name, "DEFAULT"); +} From 5d569f56a7dae9057ae84e8e18ca13a5a49c6063 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 6 May 2020 14:40:05 +0800 Subject: [PATCH 062/374] ISSUES-4006 support convert mysql create to clickhouse create --- src/CMakeLists.txt | 1 + src/Databases/DatabaseFactory.cpp | 49 ++++- ...eMySQL.cpp => DatabaseConnectionMySQL.cpp} | 48 ++--- ...abaseMySQL.h => DatabaseConnectionMySQL.h} | 8 +- .../MySQL/DatabaseMaterializeMySQL.cpp | 204 +++++++++--------- .../MySQL/DatabaseMaterializeMySQL.h | 62 +++--- src/Databases/MySQL/createMySQLDatabase.cpp | 84 -------- src/Databases/MySQL/createMySQLDatabase.h | 19 -- .../MySQL/CreateQueryConvertVisitor.cpp | 133 ++++++++++++ .../MySQL/CreateQueryConvertVisitor.h | 48 +++++ .../gtest_create_query_convert_visitor.cpp | 154 +++++++++++++ src/Parsers/MySQL/ASTDeclareColumn.cpp | 18 ++ src/Parsers/MySQL/ASTDeclareOption.cpp | 2 +- 13 files changed, 567 insertions(+), 263 deletions(-) rename src/Databases/MySQL/{DatabaseMySQL.cpp => DatabaseConnectionMySQL.cpp} (89%) rename src/Databases/MySQL/{DatabaseMySQL.h => DatabaseConnectionMySQL.h} (93%) delete mode 100644 src/Databases/MySQL/createMySQLDatabase.cpp delete mode 100644 src/Databases/MySQL/createMySQLDatabase.h create mode 100644 src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp create mode 100644 src/Interpreters/MySQL/CreateQueryConvertVisitor.h create mode 100644 src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 74f0a03408e..7f8359c4c93 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -147,6 +147,7 @@ add_object_library(clickhouse_databases Databases) add_object_library(clickhouse_databases_mysql Databases/MySQL) add_object_library(clickhouse_disks Disks) add_object_library(clickhouse_interpreters Interpreters) +add_object_library(clickhouse_interpreters_mysql Interpreters/MySQL) add_object_library(clickhouse_interpreters_clusterproxy Interpreters/ClusterProxy) add_object_library(clickhouse_columns Columns) add_object_library(clickhouse_storages Storages) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 1887317c05d..c6b79a24ff1 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -5,13 +5,11 @@ #include #include #include -#include -#include -#include -#include #include #include -#include +#include +#include +#include #include #include @@ -19,6 +17,13 @@ # include "config_core.h" #endif +#if USE_MYSQL +# include +# include +# include +# include +#endif + namespace DB { @@ -89,8 +94,40 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String return std::make_shared(database_name, context); #if USE_MYSQL + else if (engine_name == "MySQL") - return createMySQLDatabase(database_name, metadata_path, engine_define, context); + { + const ASTFunction * engine = engine_define->engine; + if (!engine->arguments || engine->arguments->children.size() != 4) + throw Exception( + "MySQL Database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.", + ErrorCodes::BAD_ARGUMENTS); + + ASTs & arguments = engine->arguments->children; + arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], context); + + const auto & host_name_and_port = safeGetLiteralValue(arguments[0], "MySQL"); + const auto & mysql_database_name = safeGetLiteralValue(arguments[1], "MySQL"); + const auto & mysql_user_name = safeGetLiteralValue(arguments[2], "MySQL"); + const auto & mysql_user_password = safeGetLiteralValue(arguments[3], "MySQL"); + + try + { + const auto & [remote_host_name, remote_port] = parseAddress(host_name_and_port, 3306); + auto mysql_pool = mysqlxx::Pool(mysql_database_name, remote_host_name, mysql_user_name, mysql_user_password, remote_port); + + /*if (materializeMySQLDatabase(define->settings)) + return std::make_shared( + context, database_name, metadata_path, define, mysql_database_name, std::move(mysql_pool));*/ + + return std::make_shared(context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_pool)); + } + catch (...) + { + const auto & exception_message = getCurrentExceptionMessage(true); + throw Exception("Cannot create MySQL database, because " + exception_message, ErrorCodes::CANNOT_CREATE_DATABASE); + } + } #endif else if (engine_name == "Lazy") diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp similarity index 89% rename from src/Databases/MySQL/DatabaseMySQL.cpp rename to src/Databases/MySQL/DatabaseConnectionMySQL.cpp index ff12446331a..7f6fc59c8ed 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -9,7 +9,7 @@ # include # include # include -# include +# include # include # include # include @@ -60,7 +60,7 @@ static String toQueryStringWithQuote(const std::vector & quote_list) return quote_list_query.str(); } -DatabaseMySQL::DatabaseMySQL( +DatabaseConnectionMySQL::DatabaseConnectionMySQL( const Context & global_context_, const String & database_name_, const String & metadata_path_, const ASTStorage * database_engine_define_, const String & database_name_in_mysql_, mysqlxx::Pool && pool) : IDatabase(database_name_) @@ -73,7 +73,7 @@ DatabaseMySQL::DatabaseMySQL( empty(); /// test database is works fine. } -bool DatabaseMySQL::empty() const +bool DatabaseConnectionMySQL::empty() const { std::lock_guard lock(mutex); @@ -89,7 +89,7 @@ bool DatabaseMySQL::empty() const return true; } -DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseConnectionMySQL::getTablesIterator(const FilterByNameFunction & filter_by_table_name) { Tables tables; std::lock_guard lock(mutex); @@ -103,12 +103,12 @@ DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const Context &, cons return std::make_unique(tables, database_name); } -bool DatabaseMySQL::isTableExist(const String & name, const Context &) const +bool DatabaseConnectionMySQL::isTableExist(const String & name) const { return bool(tryGetTable(name, global_context)); } -StoragePtr DatabaseMySQL::tryGetTable(const String & mysql_table_name, const Context &) const +StoragePtr DatabaseConnectionMySQL::tryGetTable(const String & mysql_table_name) const { std::lock_guard lock(mutex); @@ -156,7 +156,7 @@ static ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr return create_table_query; } -ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const +ASTPtr DatabaseConnectionMySQL::getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const { std::lock_guard lock(mutex); @@ -173,7 +173,7 @@ ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, const C return getCreateQueryFromStorage(local_tables_cache[table_name].second, database_engine_define); } -time_t DatabaseMySQL::getObjectMetadataModificationTime(const String & table_name) const +time_t DatabaseConnectionMySQL::getObjectMetadataModificationTime(const String & table_name) const { std::lock_guard lock(mutex); @@ -185,7 +185,7 @@ time_t DatabaseMySQL::getObjectMetadataModificationTime(const String & table_nam return time_t(local_tables_cache[table_name].first); } -ASTPtr DatabaseMySQL::getCreateDatabaseQuery() const +ASTPtr DatabaseConnectionMySQL::getCreateDatabaseQuery() const { const auto & create_query = std::make_shared(); create_query->database = getDatabaseName(); @@ -193,7 +193,7 @@ ASTPtr DatabaseMySQL::getCreateDatabaseQuery() const return create_query; } -void DatabaseMySQL::fetchTablesIntoLocalCache() const +void DatabaseConnectionMySQL::fetchTablesIntoLocalCache() const { const auto & tables_with_modification_time = fetchTablesWithModificationTime(); @@ -201,7 +201,7 @@ void DatabaseMySQL::fetchTablesIntoLocalCache() const fetchLatestTablesStructureIntoCache(tables_with_modification_time); } -void DatabaseMySQL::destroyLocalCacheExtraTables(const std::map & tables_with_modification_time) const +void DatabaseConnectionMySQL::destroyLocalCacheExtraTables(const std::map & tables_with_modification_time) const { for (auto iterator = local_tables_cache.begin(); iterator != local_tables_cache.end();) { @@ -215,7 +215,7 @@ void DatabaseMySQL::destroyLocalCacheExtraTables(const std::map } } -void DatabaseMySQL::fetchLatestTablesStructureIntoCache(const std::map &tables_modification_time) const +void DatabaseConnectionMySQL::fetchLatestTablesStructureIntoCache(const std::map &tables_modification_time) const { std::vector wait_update_tables_name; for (const auto & table_modification_time : tables_modification_time) @@ -248,7 +248,7 @@ void DatabaseMySQL::fetchLatestTablesStructureIntoCache(const std::map DatabaseMySQL::fetchTablesWithModificationTime() const +std::map DatabaseConnectionMySQL::fetchTablesWithModificationTime() const { Block tables_status_sample_block { @@ -279,7 +279,7 @@ std::map DatabaseMySQL::fetchTablesWithModificationTime() const return tables_with_modification_time; } -std::map DatabaseMySQL::fetchTablesColumnsList(const std::vector & tables_name) const +std::map DatabaseConnectionMySQL::fetchTablesColumnsList(const std::vector & tables_name) const { std::map tables_and_columns; @@ -328,7 +328,7 @@ std::map DatabaseMySQL::fetchTablesColumnsList(const return tables_and_columns; } -void DatabaseMySQL::shutdown() +void DatabaseConnectionMySQL::shutdown() { std::map tables_snapshot; { @@ -343,12 +343,12 @@ void DatabaseMySQL::shutdown() local_tables_cache.clear(); } -void DatabaseMySQL::drop(const Context & /*context*/) +void DatabaseConnectionMySQL::drop(const Context & /*context*/) { Poco::File(getMetadataPath()).remove(true); } -void DatabaseMySQL::cleanOutdatedTables() +void DatabaseConnectionMySQL::cleanOutdatedTables() { setThreadName("MySQLDBCleaner"); @@ -374,7 +374,7 @@ void DatabaseMySQL::cleanOutdatedTables() } } -void DatabaseMySQL::attachTable(const String & table_name, const StoragePtr & storage, const String &) +void DatabaseConnectionMySQL::attachTable(const String & table_name, const StoragePtr & storage, const String &) { std::lock_guard lock{mutex}; @@ -397,7 +397,7 @@ void DatabaseMySQL::attachTable(const String & table_name, const StoragePtr & st remove_flag.remove(); } -StoragePtr DatabaseMySQL::detachTable(const String & table_name) +StoragePtr DatabaseConnectionMySQL::detachTable(const String & table_name) { std::lock_guard lock{mutex}; @@ -413,12 +413,12 @@ StoragePtr DatabaseMySQL::detachTable(const String & table_name) return local_tables_cache[table_name].second; } -String DatabaseMySQL::getMetadataPath() const +String DatabaseConnectionMySQL::getMetadataPath() const { return metadata_path; } -void DatabaseMySQL::loadStoredObjects(Context &, bool) +void DatabaseConnectionMySQL::loadStoredObjects(Context &, bool) { std::lock_guard lock{mutex}; @@ -435,7 +435,7 @@ void DatabaseMySQL::loadStoredObjects(Context &, bool) } } -void DatabaseMySQL::dropTable(const Context &, const String & table_name, bool /*no_delay*/) +void DatabaseConnectionMySQL::dropTable(const Context &, const String & table_name, bool /*no_delay*/) { std::lock_guard lock{mutex}; @@ -469,7 +469,7 @@ void DatabaseMySQL::dropTable(const Context &, const String & table_name, bool / table_iter->second.second->is_dropped = true; } -DatabaseMySQL::~DatabaseMySQL() +DatabaseConnectionMySQL::~DatabaseConnectionMySQL() { try { @@ -491,7 +491,7 @@ DatabaseMySQL::~DatabaseMySQL() } } -void DatabaseMySQL::createTable(const Context &, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) +void DatabaseConnectionMySQL::createTable(const Context &, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) { const auto & create = create_query->as(); diff --git a/src/Databases/MySQL/DatabaseMySQL.h b/src/Databases/MySQL/DatabaseConnectionMySQL.h similarity index 93% rename from src/Databases/MySQL/DatabaseMySQL.h rename to src/Databases/MySQL/DatabaseConnectionMySQL.h index 5f45713537c..ba50facc81a 100644 --- a/src/Databases/MySQL/DatabaseMySQL.h +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.h @@ -19,12 +19,12 @@ class Context; * It doesn't make any manipulations with filesystem. * All tables are created by calling code after real-time pull-out structure from remote MySQL */ -class DatabaseMySQL final : public IDatabase +class DatabaseConnectionMySQL final : public IDatabase { public: - ~DatabaseMySQL() override; + ~DatabaseConnectionMySQL() override; - DatabaseMySQL( + DatabaseConnectionMySQL( const Context & global_context, const String & database_name, const String & metadata_path, const ASTStorage * database_engine_define, const String & database_name_in_mysql, mysqlxx::Pool && pool); @@ -91,7 +91,7 @@ private: void fetchLatestTablesStructureIntoCache(const std::map & tables_modification_time) const; - ThreadFromGlobalPool thread{&DatabaseMySQL::cleanOutdatedTables, this}; + ThreadFromGlobalPool thread{&DatabaseConnectionMySQL::cleanOutdatedTables, this}; }; } diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 65e5a76ae98..9fdf2c037ea 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -1,97 +1,107 @@ -//#include -// -//#include -//#include -//#include -//#include -//#include -//#include -//#include -// -//namespace DB -//{ -// -//static MasterStatusInfo fetchMasterStatus(const mysqlxx::PoolWithFailover::Entry & connection) -//{ -// Block header -// { -// {std::make_shared(), "File"}, -// {std::make_shared(), "Position"}, -// {std::make_shared(), "Binlog_Do_DB"}, -// {std::make_shared(), "Binlog_Ignore_DB"}, -// {std::make_shared(), "Executed_Gtid_Set"}, -// }; -// -// MySQLBlockInputStream input(connection, "SHOW MASTER STATUS;", header, DEFAULT_BLOCK_SIZE); -// Block master_status = input.read(); -// -// if (!master_status || master_status.rows() != 1) -// throw Exception("Unable to get master status from MySQL.", ErrorCodes::LOGICAL_ERROR); -// -// return MasterStatusInfo -// { -// (*master_status.getByPosition(0).column)[0].safeGet(), -// (*master_status.getByPosition(1).column)[0].safeGet(), -// (*master_status.getByPosition(2).column)[0].safeGet(), -// (*master_status.getByPosition(3).column)[0].safeGet(), -// (*master_status.getByPosition(4).column)[0].safeGet() -// }; -//} -// -//static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entry & connection, const std::string & database) -//{ -// Block header{{std::make_shared(), "table_name"}}; -// String query = "SELECT TABLE_NAME AS table_name FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA = " + quoteString(database); -// -// std::vector tables_in_db; -// MySQLBlockInputStream input(connection, query, header, DEFAULT_BLOCK_SIZE); -// -// while (Block block = input.read()) -// { -// tables_in_db.reserve(tables_in_db.size() + block.rows()); -// for (size_t index = 0; index < block.rows(); ++index) -// tables_in_db.emplace_back((*block.getByPosition(0).column)[index].safeGet()); -// } -// -// return tables_in_db; -//} -// -//DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( -// const Context & context, const String & database_name_, const String & metadata_path_, -// const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_) -// : IDatabase(database_name_) -// , global_context(context.getGlobalContext()), metadata_path(metadata_path_) -// , database_engine_define(database_engine_define_->clone()), mysql_database_name(mysql_database_name_), pool(std::move(pool_)) -//{ -// try -// { -// mysqlxx::PoolWithFailover::Entry connection = pool.get(); -// -// connection->query("FLUSH TABLES;").execute(); -// connection->query("FLUSH TABLES WITH READ LOCK;").execute(); -// -// MasterStatusInfo master_status = fetchMasterStatus(connection); -// connection->query("SET SESSION TRANSACTION ISOLATION LEVEL REPEATABLE READ;").execute(); -// connection->query("START TRANSACTION /*!40100 WITH CONSISTENT SNAPSHOT */;").execute(); -// -// std::vector tables_in_db = fetchTablesInDB(connection, mysql_database_name); -// connection->query("UNLOCK TABLES;"); -// -// for (const auto & dumping_table_name : tables_in_db) -// { -// /// TODO: 查询表结构, 根据不同的模式创建对应的表(暂时只支持多version即可) -//// connection->query("SHOW CREATE TABLE " + doubleQuoteString()) -// MySQLBlockInputStream input( -// "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(dumping_table_name)); -// /// TODO: 查询所有数据写入对应表中(全量dump) -// /// TODO: 启动slave, 监听事件 -// } -// } -// catch (...) -// { -// throw; -// } -//} -// -// -//} +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#if USE_MYSQL + +#include + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +static MasterStatusInfo fetchMasterStatus(const mysqlxx::PoolWithFailover::Entry & connection) +{ + Block header + { + {std::make_shared(), "File"}, + {std::make_shared(), "Position"}, + {std::make_shared(), "Binlog_Do_DB"}, + {std::make_shared(), "Binlog_Ignore_DB"}, + {std::make_shared(), "Executed_Gtid_Set"}, + }; + + MySQLBlockInputStream input(connection, "SHOW MASTER STATUS;", header, DEFAULT_BLOCK_SIZE); + Block master_status = input.read(); + + if (!master_status || master_status.rows() != 1) + throw Exception("Unable to get master status from MySQL.", ErrorCodes::LOGICAL_ERROR); + + return MasterStatusInfo + { + (*master_status.getByPosition(0).column)[0].safeGet(), + (*master_status.getByPosition(1).column)[0].safeGet(), + (*master_status.getByPosition(2).column)[0].safeGet(), + (*master_status.getByPosition(3).column)[0].safeGet(), + (*master_status.getByPosition(4).column)[0].safeGet() + }; +} + +static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entry & connection, const std::string & database) +{ + Block header{{std::make_shared(), "table_name"}}; + String query = "SELECT TABLE_NAME AS table_name FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA = " + quoteString(database); + + std::vector tables_in_db; + MySQLBlockInputStream input(connection, query, header, DEFAULT_BLOCK_SIZE); + + while (Block block = input.read()) + { + tables_in_db.reserve(tables_in_db.size() + block.rows()); + for (size_t index = 0; index < block.rows(); ++index) + tables_in_db.emplace_back((*block.getByPosition(0).column)[index].safeGet()); + } + + return tables_in_db; +} + +DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( + const Context & context, const String & database_name_, const String & metadata_path_, + const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_) + : DatabaseOrdinary(database_name_, metadata_path_, context) + /*, global_context(context.getGlobalContext()), metadata_path(metadata_path_)*/ + , database_engine_define(database_engine_define_->clone()), mysql_database_name(mysql_database_name_), pool(std::move(pool_)) +{ + try + { + mysqlxx::PoolWithFailover::Entry connection = pool.get(); + + connection->query("FLUSH TABLES;").execute(); + connection->query("FLUSH TABLES WITH READ LOCK;").execute(); + + MasterStatusInfo master_status = fetchMasterStatus(connection); + connection->query("SET SESSION TRANSACTION ISOLATION LEVEL REPEATABLE READ;").execute(); + connection->query("START TRANSACTION /*!40100 WITH CONSISTENT SNAPSHOT */;").execute(); + + std::vector tables_in_db = fetchTablesInDB(connection, mysql_database_name); + connection->query("UNLOCK TABLES;").execute(); + + for (const auto & dumping_table_name : tables_in_db) + { + /// TODO: 查询表结构, 根据不同的模式创建对应的表(暂时只支持多version即可) + connection->query("SHOW CREATE TABLE " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(dumping_table_name)) + .execute(); + MySQLBlockInputStream input( + "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(dumping_table_name)); + copyData(input, output); + /// TODO: 查询所有数据写入对应表中(全量dump) + /// TODO: 启动slave, 监听事件 + } + } + catch (...) + { + throw; + } +} + + +} + +#endif diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h index bb7e6193451..136df735c79 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -1,30 +1,36 @@ -//#pragma once -// -//#include -//#include -//#include -//#include -//#include -// -//namespace DB -//{ -// -//class DatabaseMaterializeMySQL : public IDatabase -//{ -//public: -// DatabaseMaterializeMySQL( -// const Context & context, const String & database_name_, const String & metadata_path_, -// const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_); -// -// String getEngineName() const override { return "MySQL"; } -// -//private: +#pragma once + +#include "config_core.h" +#if USE_MYSQL + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class DatabaseMaterializeMySQL : public DatabaseOrdinary +{ +public: + DatabaseMaterializeMySQL( + const Context & context, const String & database_name_, const String & metadata_path_, + const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_); + + String getEngineName() const override { return "MySQL"; } + +private: // const Context & global_context; // String metadata_path; -// ASTPtr database_engine_define; -// String mysql_database_name; -// -// mutable mysqlxx::Pool pool; -//}; -// -//} + ASTPtr database_engine_define; + String mysql_database_name; + + mutable mysqlxx::Pool pool; +}; + +} + +#endif diff --git a/src/Databases/MySQL/createMySQLDatabase.cpp b/src/Databases/MySQL/createMySQLDatabase.cpp deleted file mode 100644 index 24417d83791..00000000000 --- a/src/Databases/MySQL/createMySQLDatabase.cpp +++ /dev/null @@ -1,84 +0,0 @@ -#include - -#if USE_MYSQL - -#include -#include -#include -#include -#include - -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int CANNOT_CREATE_DATABASE; -} - -static inline String safeGetLiteralValue(const ASTPtr & ast) -{ - if (!ast || !ast->as()) - throw Exception("Database engine MySQL requested literal argument.", ErrorCodes::BAD_ARGUMENTS); - - return ast->as()->value.safeGet(); -} - -/*static inline bool materializeMySQLDatabase(const ASTSetQuery * settings) -{ - if (!settings || settings->changes.empty()) - return false; - - for (const auto & change : settings->changes) - { - if (change.name == "materialize_data") - { - if (change.value.getType() == Field::Types::String) - return change.value.safeGet() == "true"; ///TODO: ignore case - } - - } - return false; -}*/ - -DatabasePtr createMySQLDatabase(const String & database_name, const String & metadata_path, const ASTStorage * define, Context & context) -{ - const ASTFunction * engine = define->engine; - if (!engine->arguments || engine->arguments->children.size() != 4) - throw Exception( "MySQL Database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.", - ErrorCodes::BAD_ARGUMENTS); - - ASTs & arguments = engine->arguments->children; - arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], context); - - const auto & host_name_and_port = safeGetLiteralValue(arguments[0]); - const auto & mysql_database_name = safeGetLiteralValue(arguments[1]); - const auto & mysql_user_name = safeGetLiteralValue(arguments[2]); - const auto & mysql_user_password = safeGetLiteralValue(arguments[3]); - - try - { - const auto & [remote_host_name, remote_port] = parseAddress(host_name_and_port, 3306); - auto mysql_pool = mysqlxx::Pool(mysql_database_name, remote_host_name, mysql_user_name, mysql_user_password, remote_port); - - /*if (materializeMySQLDatabase(define->settings)) - return std::make_shared( - context, database_name, metadata_path, define, mysql_database_name, std::move(mysql_pool));*/ - - return std::make_shared(context, database_name, metadata_path, define, mysql_database_name, std::move(mysql_pool)); - } - catch (...) - { - const auto & exception_message = getCurrentExceptionMessage(true); - throw Exception("Cannot create MySQL database, because " + exception_message, ErrorCodes::CANNOT_CREATE_DATABASE); - } -} - -} - -#endif diff --git a/src/Databases/MySQL/createMySQLDatabase.h b/src/Databases/MySQL/createMySQLDatabase.h deleted file mode 100644 index f85e96428fe..00000000000 --- a/src/Databases/MySQL/createMySQLDatabase.h +++ /dev/null @@ -1,19 +0,0 @@ -#pragma once - -#if !defined(ARCADIA_BUILD) -# include "config_core.h" -#endif - -#if USE_MYSQL - -#include -#include - -namespace DB -{ - -DatabasePtr createMySQLDatabase(const String & database_name, const String & metadata_path, const ASTStorage * define, Context & context); - -} - -#endif diff --git a/src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp b/src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp new file mode 100644 index 00000000000..9c2fe24d48b --- /dev/null +++ b/src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp @@ -0,0 +1,133 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_TYPE; + extern const int NOT_IMPLEMENTED; + extern const int EMPTY_LIST_OF_COLUMNS_PASSED; +} + +namespace MySQLVisitor +{ + +void CreateQueryMatcher::visit(ASTPtr & ast, Data & data) +{ + if (auto * t = ast->as()) + visit(*t, ast, data); +} +void CreateQueryMatcher::visit(ASTCreateQuery & create, ASTPtr & /*ast*/, Data & data) +{ + if (create.like_table) + throw Exception("Cannot convert create like statement to ClickHouse SQL", ErrorCodes::NOT_IMPLEMENTED); + + data.out << "CREATE TABLE " << (create.if_not_exists ? "IF NOT EXISTS" : "") + << (create.database.empty() ? "" : backQuoteIfNeed(create.database) + ".") << backQuoteIfNeed(create.table) << "("; + + if (create.columns_list) + visitColumns(*create.columns_list->as(), create.columns_list, data); + + data.out << ") ENGINE = MergeTree()"; +} + +void CreateQueryMatcher::visitColumns(ASTCreateDefines & create_defines, ASTPtr & /*ast*/, Data & data) +{ + if (!create_defines.columns || create_defines.columns->children.empty()) + throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); + + bool is_first = true; + for (auto & column : create_defines.columns->children) + { + if (!is_first) + data.out << ","; + + is_first = false; + visitColumns(*column->as(), column, data); + } +} + +static String convertDataType(const String & type_name, const ASTPtr & arguments, bool is_unsigned, bool /*is_national*/) +{ + if (type_name == "TINYINT") + return is_unsigned ? "UInt8" : "Int8"; + else if (type_name == "BOOL" || type_name == "BOOLEAN") + return "UInt8"; + else if (type_name == "SMALLINT") + return is_unsigned ? "UInt16" : "Int16"; + else if (type_name == "INT" || type_name == "MEDIUMINT" || type_name == "INTEGER") + return is_unsigned ? "UInt32" : "Int32"; + else if (type_name == "BIGINT") + return is_unsigned ? "UInt64" : "Int64"; + else if (type_name == "FLOAT") + return "Float32"; + else if (type_name == "DOUBLE" || type_name == "PRECISION" || type_name == "REAL") + return "Float64"; + else if (type_name == "DECIMAL" || type_name == "DEC" || type_name == "NUMERIC" || type_name == "FIXED") + return arguments ? "Decimal(" + queryToString(arguments) + ")" : "Decimal(10, 0)"; + + if (type_name == "DATE") + return "Date"; + else if (type_name == "DATETIME" || type_name == "TIMESTAMP") + return "DateTime"; + else if (type_name == "TIME") + return "DateTime64"; + else if (type_name == "YEAR") + return "Int16"; + + if (type_name == "BINARY") + return arguments ? "FixedString(" + queryToString(arguments) + ")" : "FixedString(1)"; + + return "String"; +} + +void CreateQueryMatcher::visitColumns(const ASTDeclareColumn & declare_column, ASTPtr &, Data & data) +{ + data.out << declare_column.name << " "; + + if (!declare_column.data_type) + throw Exception("Missing type in definition of column.", ErrorCodes::UNKNOWN_TYPE); + + bool is_unsigned = false; + bool is_nullable = true; + bool is_national = false; + if (declare_column.column_options) + { + if (ASTDeclareOptions * options = declare_column.column_options->as()) + { + if (options->changes.count("is_null")) + is_nullable = options->changes["is_null"]->as()->value.safeGet(); + + if (options->changes.count("is_national")) + is_national = options->changes["is_national"]->as()->value.safeGet(); + + if (options->changes.count("is_unsigned")) + is_unsigned = options->changes["is_unsigned"]->as()->value.safeGet(); + else if (options->changes.count("zero_fill")) + is_unsigned = options->changes["zero_fill"]->as()->value.safeGet(); + } + } + + if (ASTFunction * function = declare_column.data_type->as()) + data.out << (is_nullable ? "Nullable(" : "") + << convertDataType(Poco::toUpper(function->name), function->arguments, is_unsigned, is_national) + << (is_nullable ? ")" : ""); + + if (ASTIdentifier * identifier = declare_column.data_type->as()) + data.out << (is_nullable ? "Nullable(" : "") << convertDataType(Poco::toUpper(identifier->name), ASTPtr{}, is_unsigned, is_national) + << (is_nullable ? ")" : ""); +} + +} + +} diff --git a/src/Interpreters/MySQL/CreateQueryConvertVisitor.h b/src/Interpreters/MySQL/CreateQueryConvertVisitor.h new file mode 100644 index 00000000000..2c0c9ad6d7f --- /dev/null +++ b/src/Interpreters/MySQL/CreateQueryConvertVisitor.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +using namespace MySQLParser; + +namespace MySQLVisitor +{ + +/// Convert MySQL CREATE query to https://clickhouse.tech/docs/en/sql-reference/statements/create/ +class CreateQueryMatcher +{ +public: + using Visitor = InDepthNodeVisitor; + + struct Data + { + /// SETTINGS + WriteBuffer & out; + std::string declare_columns; + }; + + static void visit(ASTPtr & ast, Data & data); + + static bool needChildVisit(ASTPtr &, const ASTPtr &) { return false; } +private: + static void visit(ASTCreateQuery & create, ASTPtr & ast, Data &); + + static void visitColumns(ASTCreateDefines & create_defines, ASTPtr & ast, Data & data); + + static void visitColumns(const ASTDeclareColumn & declare_column, ASTPtr & ast, Data & data); + +// static void visitColumns(const ASTFunction & declare_column, ASTPtr & ast, Data & data); +// static void visit(ASTTableJoin & join, const ASTPtr & ast, Data &); + +}; + +using CreateQueryConvertVisitor = CreateQueryMatcher::Visitor; + +} + +} diff --git a/src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp b/src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp new file mode 100644 index 00000000000..6a8ae26e4c4 --- /dev/null +++ b/src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp @@ -0,0 +1,154 @@ +#include +#include +#include +#include +#include + +using namespace DB; +using namespace MySQLParser; +using namespace MySQLVisitor; + +TEST(CreateQueryConvert, SimpleNumbersType) +{ + ParserCreateQuery p_create_query; + String input = "CREATE TABLE test(a tinyint, b SMALLINT, c MEDIUMINT, d INT, e INTEGER, f BIGINT, g DECIMAL, h DEC, i NUMERIC, j " + "FIXED, k FLOAT, l DOUBLE, m DOUBLE PRECISION, n REAL)"; + ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); + + WriteBufferFromOwnString out; + CreateQueryConvertVisitor::Data data{.out = out}; + CreateQueryConvertVisitor visitor(data); + visitor.visit(ast); + + EXPECT_EQ( + out.str(), + "CREATE TABLE test(a Nullable(Int8),b Nullable(Int16),c Nullable(Int32),d Nullable(Int32),e Nullable(Int32),f Nullable(Int64),g " + "Nullable(Decimal(10, 0)),h Nullable(Decimal(10, 0)),i Nullable(Decimal(10, 0)),j Nullable(Decimal(10, 0)),k Nullable(Float32),l " + "Nullable(Float64),m Nullable(Float64),n Nullable(Float64)) ENGINE = MergeTree()"); +} + +TEST(CreateQueryConvert, NumbersTypeWithLength) +{ + ParserCreateQuery p_create_query; + String input = "CREATE TABLE test(a tinyint(1), b SMALLINT(1), c MEDIUMINT(1), d INT(1), e INTEGER(1), f BIGINT(1), g DECIMAL(1), h DEC(1, 2), i NUMERIC(3, 4), j " + "FIXED(5, 6), k FLOAT(1), l DOUBLE(1, 2), m DOUBLE PRECISION(3, 4), n REAL(5, 6))"; + ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); + + WriteBufferFromOwnString out; + CreateQueryConvertVisitor::Data data{.out = out}; + CreateQueryConvertVisitor visitor(data); + visitor.visit(ast); + + EXPECT_EQ( + out.str(), + "CREATE TABLE test(a Nullable(Int8),b Nullable(Int16),c Nullable(Int32),d Nullable(Int32),e Nullable(Int32),f Nullable(Int64),g " + "Nullable(Decimal(1)),h Nullable(Decimal(1, 2)),i Nullable(Decimal(3, 4)),j Nullable(Decimal(5, 6)),k Nullable(Float32),l " + "Nullable(Float64),m Nullable(Float64),n Nullable(Float64)) ENGINE = MergeTree()"); +} + +TEST(CreateQueryConvert, NumbersTypeWithUnsigned) +{ + ParserCreateQuery p_create_query; + String input = "CREATE TABLE test(a tinyint UNSIGNED, b SMALLINT(1) UNSIGNED, c MEDIUMINT(1) UNSIGNED, d INT(1) UNSIGNED, e " + "INTEGER(1), f BIGINT(1) UNSIGNED, g DECIMAL(1) UNSIGNED, h DEC(1, 2) UNSIGNED, i NUMERIC(3, 4) UNSIGNED, j FIXED(5, 6) " + "UNSIGNED, k FLOAT(1) UNSIGNED, l DOUBLE(1, 2) UNSIGNED, m DOUBLE PRECISION(3, 4) UNSIGNED, n REAL(5, 6) UNSIGNED)"; + ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); + + WriteBufferFromOwnString out; + CreateQueryConvertVisitor::Data data{.out = out}; + CreateQueryConvertVisitor visitor(data); + visitor.visit(ast); + + EXPECT_EQ( + out.str(), + "CREATE TABLE test(a Nullable(UInt8),b Nullable(UInt16),c Nullable(UInt32),d Nullable(UInt32),e Nullable(Int32),f " + "Nullable(UInt64),g Nullable(Decimal(1)),h Nullable(Decimal(1, 2)),i Nullable(Decimal(3, 4)),j Nullable(Decimal(5, 6)),k " + "Nullable(Float32),l Nullable(Float64),m Nullable(Float64),n Nullable(Float64)) ENGINE = MergeTree()"); +} + +TEST(CreateQueryConvert, NumbersTypeWithNotNull) +{ + ParserCreateQuery p_create_query; + String input = "CREATE TABLE test(a tinyint NOT NULL, b SMALLINT(1) NOT NULL, c MEDIUMINT(1) NOT NULL, d INT(1) NOT NULL, e " + "INTEGER(1), f BIGINT(1) NOT NULL, g DECIMAL(1) NOT NULL, h DEC(1, 2) NOT NULL, i NUMERIC(3, 4) NOT NULL, j FIXED(5, 6) " + "NOT NULL, k FLOAT(1) NOT NULL, l DOUBLE(1, 2) NOT NULL, m DOUBLE PRECISION(3, 4) NOT NULL, n REAL(5, 6) NOT NULL)"; + ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); + + WriteBufferFromOwnString out; + CreateQueryConvertVisitor::Data data{.out = out}; + CreateQueryConvertVisitor visitor(data); + visitor.visit(ast); + + EXPECT_EQ( + out.str(), + "CREATE TABLE test(a Int8,b Int16,c Int32,d Int32,e Nullable(Int32),f Int64,g Decimal(1),h Decimal(1, 2),i Decimal(3, 4),j " + "Decimal(5, 6),k Float32,l Float64,m Float64,n Float64) ENGINE = MergeTree()"); +} + +TEST(CreateQueryConvert, NumbersTypeWithZeroFill) +{ + ParserCreateQuery p_create_query; + String input = "CREATE TABLE test(a tinyint ZEROFILL, b SMALLINT(1) ZEROFILL, c MEDIUMINT(1) ZEROFILL, d INT(1) ZEROFILL, e " + "INTEGER(1), f BIGINT(1) ZEROFILL, g DECIMAL(1) ZEROFILL, h DEC(1, 2) ZEROFILL, i NUMERIC(3, 4) ZEROFILL, j FIXED(5, 6) " + "ZEROFILL, k FLOAT(1) ZEROFILL, l DOUBLE(1, 2) ZEROFILL, m DOUBLE PRECISION(3, 4) ZEROFILL, n REAL(5, 6) ZEROFILL)"; + ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); + + WriteBufferFromOwnString out; + CreateQueryConvertVisitor::Data data{.out = out}; + CreateQueryConvertVisitor visitor(data); + visitor.visit(ast); + + EXPECT_EQ( + out.str(), + "CREATE TABLE test(a Nullable(UInt8),b Nullable(UInt16),c Nullable(UInt32),d Nullable(UInt32),e Nullable(Int32),f " + "Nullable(UInt64),g Nullable(Decimal(1)),h Nullable(Decimal(1, 2)),i Nullable(Decimal(3, 4)),j Nullable(Decimal(5, 6)),k " + "Nullable(Float32),l Nullable(Float64),m Nullable(Float64),n Nullable(Float64)) ENGINE = MergeTree()"); +} + +TEST(CreateQueryConvert, SimpleDateTimesType) +{ + ParserCreateQuery p_create_query; + String input = "CREATE TABLE test(a DATE, b DATETIME, c TIMESTAMP, d TIME, e year)"; + ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); + + WriteBufferFromOwnString out; + CreateQueryConvertVisitor::Data data{.out = out}; + CreateQueryConvertVisitor visitor(data); + visitor.visit(ast); + + EXPECT_EQ( + out.str(), + "CREATE TABLE test(a Nullable(Date),b Nullable(DateTime),c Nullable(DateTime),d Nullable(DateTime64),e Nullable(Int16)) ENGINE = " + "MergeTree()"); +} + +TEST(CreateQueryConvert, DateTimeTypesWithLength) +{ + ParserCreateQuery p_create_query; + String input = "CREATE TABLE test(a DATE, b DATETIME(1), c TIMESTAMP(1), d TIME(1), e year(4))"; + ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); + + WriteBufferFromOwnString out; + CreateQueryConvertVisitor::Data data{.out = out}; + CreateQueryConvertVisitor visitor(data); + visitor.visit(ast); + + EXPECT_EQ( + out.str(), + "CREATE TABLE test(a Nullable(Date),b Nullable(DateTime),c Nullable(DateTime),d Nullable(DateTime64),e Nullable(Int16)) ENGINE = " + "MergeTree()"); +} + +TEST(CreateQueryConvert, DateTimeTypesWithNotNull) +{ + ParserCreateQuery p_create_query; + String input = "CREATE TABLE test(a DATE NOT NULL, b DATETIME(1) NOT NULL, c TIMESTAMP(1) NOT NULL, d TIME(1) NOT NULL, e year(4) NOT NULL)"; + ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); + + WriteBufferFromOwnString out; + CreateQueryConvertVisitor::Data data{.out = out}; + CreateQueryConvertVisitor visitor(data); + visitor.visit(ast); + + EXPECT_EQ(out.str(), "CREATE TABLE test(a Date,b DateTime,c DateTime,d DateTime64,e Int16) ENGINE = MergeTree()"); +} diff --git a/src/Parsers/MySQL/ASTDeclareColumn.cpp b/src/Parsers/MySQL/ASTDeclareColumn.cpp index 295818a66c5..91dede7a9a7 100644 --- a/src/Parsers/MySQL/ASTDeclareColumn.cpp +++ b/src/Parsers/MySQL/ASTDeclareColumn.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -38,6 +39,7 @@ bool ParserDeclareColumn::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte ASTPtr column_name; ASTPtr column_data_type; ASTPtr column_options; + bool is_national = false; ParserExpression p_expression; ParserIdentifier p_identifier; @@ -45,12 +47,25 @@ bool ParserDeclareColumn::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte if (!p_identifier.parse(pos, column_name, expected)) return false; + if (ParserKeyword("NATIONAL").checkWithoutMoving(pos, expected)) + is_national = true; + else if (ParserKeyword("DOUBLE PRECISION").checkWithoutMoving(pos, expected)) + ParserKeyword("DOUBLE").ignore(pos, expected); /// hack skip DOUBLE + if (!p_expression.parse(pos, column_data_type, expected)) return false; if (!parseColumnDeclareOptions(pos, column_options, expected)) return false; + if (is_national) + { + if (!column_options) + column_options = std::make_shared(); + column_options->as()->changes.insert( + std::make_pair("is_national", std::make_shared(Field(UInt64(1))))); + } + auto declare_column = std::make_shared(); declare_column->name = column_name->as()->name; declare_column->data_type = column_data_type; @@ -69,6 +84,8 @@ bool ParserDeclareColumn::parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & { ParserDeclareOption p_non_generate_options{ { + OptionDescribe("ZEROFILL", "zero_fill", std::make_unique()), + OptionDescribe("UNSIGNED", "is_unsigned", std::make_unique()), OptionDescribe("NULL", "is_null", std::make_unique()), OptionDescribe("NOT NULL", "is_null", std::make_unique()), OptionDescribe("DEFAULT", "default", std::make_unique()), @@ -78,6 +95,7 @@ bool ParserDeclareColumn::parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & OptionDescribe("KEY", "primary_key", std::make_unique()), OptionDescribe("PRIMARY KEY", "primary_key", std::make_unique()), OptionDescribe("COMMENT", "comment", std::make_unique()), + OptionDescribe("CHARACTER SET", "charset_name", std::make_unique()), OptionDescribe("COLLATE", "collate", std::make_unique()), OptionDescribe("COLUMN_FORMAT", "column_format", std::make_unique()), OptionDescribe("STORAGE", "storage", std::make_unique()), diff --git a/src/Parsers/MySQL/ASTDeclareOption.cpp b/src/Parsers/MySQL/ASTDeclareOption.cpp index 347fd5d479b..9612d49cb12 100644 --- a/src/Parsers/MySQL/ASTDeclareOption.cpp +++ b/src/Parsers/MySQL/ASTDeclareOption.cpp @@ -58,7 +58,7 @@ bool ParserDeclareOption::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte if (!found) break; - ParserToken{TokenType::Comma}.ignore(pos, expected); +// ParserToken{TokenType::Comma}.ignore(pos, expected); } if (!changes.empty()) From bd18c1cbf89bd934d4659afc1e77afc6a15258b1 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 8 May 2020 15:36:48 +0800 Subject: [PATCH 063/374] ISSUES-4006 support dump all mysql data when create database --- src/Databases/DatabaseFactory.cpp | 5 +- .../MySQL/DatabaseMaterializeMySQL.cpp | 151 ++++++++++++++---- .../MySQL/DatabaseMaterializeMySQL.h | 14 ++ 3 files changed, 137 insertions(+), 33 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index c6b79a24ff1..cf7e93a0e1f 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -19,6 +19,7 @@ #if USE_MYSQL # include +# include # include # include # include @@ -116,9 +117,9 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String const auto & [remote_host_name, remote_port] = parseAddress(host_name_and_port, 3306); auto mysql_pool = mysqlxx::Pool(mysql_database_name, remote_host_name, mysql_user_name, mysql_user_password, remote_port); - /*if (materializeMySQLDatabase(define->settings)) + if (materializeMySQLDatabase(engine_define->settings)) return std::make_shared( - context, database_name, metadata_path, define, mysql_database_name, std::move(mysql_pool));*/ + context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_pool)); return std::make_shared(context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_pool)); } diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 9fdf2c037ea..288e99cd906 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -6,17 +6,29 @@ #include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include namespace DB { +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + static MasterStatusInfo fetchMasterStatus(const mysqlxx::PoolWithFailover::Entry & connection) { Block header @@ -69,38 +81,115 @@ DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( /*, global_context(context.getGlobalContext()), metadata_path(metadata_path_)*/ , database_engine_define(database_engine_define_->clone()), mysql_database_name(mysql_database_name_), pool(std::move(pool_)) { + /// TODO: 做简单的check, 失败即报错 +} + +String DatabaseMaterializeMySQL::getCreateQuery(const mysqlxx::Pool::Entry & connection, const String & database, const String & table_name) +{ + Block show_create_table_header{ + {std::make_shared(), "Table"}, + {std::make_shared(), "Create Table"}, + }; + + MySQLBlockInputStream show_create_table( + connection, "SHOW CREATE TABLE " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name), + show_create_table_header, DEFAULT_BLOCK_SIZE); + + Block create_query_block = show_create_table.read(); + if (!create_query_block || create_query_block.rows() != 1) + throw Exception("LOGICAL ERROR mysql show create return more rows.", ErrorCodes::LOGICAL_ERROR); + + const auto & create_query = create_query_block.getByName("Create Table").column->getDataAt(0); + + MySQLParser::ParserCreateQuery p_create_query; + ASTPtr ast = parseQuery(p_create_query, create_query.data, create_query.data + create_query.size, "", 0, 0); + + WriteBufferFromOwnString out; + MySQLVisitor::CreateQueryConvertVisitor::Data data{.out = out}; + MySQLVisitor::CreateQueryConvertVisitor visitor(data); + visitor.visit(ast); + return out.str(); +} + +void DatabaseMaterializeMySQL::tryToExecuteQuery(const String & query_to_execute) +{ + ReadBufferFromString istr(query_to_execute); + String dummy_string; + WriteBufferFromString ostr(dummy_string); + try { - mysqlxx::PoolWithFailover::Entry connection = pool.get(); - - connection->query("FLUSH TABLES;").execute(); - connection->query("FLUSH TABLES WITH READ LOCK;").execute(); - - MasterStatusInfo master_status = fetchMasterStatus(connection); - connection->query("SET SESSION TRANSACTION ISOLATION LEVEL REPEATABLE READ;").execute(); - connection->query("START TRANSACTION /*!40100 WITH CONSISTENT SNAPSHOT */;").execute(); - - std::vector tables_in_db = fetchTablesInDB(connection, mysql_database_name); - connection->query("UNLOCK TABLES;").execute(); - - for (const auto & dumping_table_name : tables_in_db) - { - /// TODO: 查询表结构, 根据不同的模式创建对应的表(暂时只支持多version即可) - connection->query("SHOW CREATE TABLE " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(dumping_table_name)) - .execute(); - MySQLBlockInputStream input( - "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(dumping_table_name)); - copyData(input, output); - /// TODO: 查询所有数据写入对应表中(全量dump) - /// TODO: 启动slave, 监听事件 - } + Context context = global_context; + context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + context.setCurrentQueryId(""); // generate random query_id + executeQuery(istr, ostr, false, context, {}); } catch (...) { + tryLogCurrentException(log, "Query " + query_to_execute + " wasn't finished successfully"); throw; } -} + LOG_DEBUG(log, "Executed query: " << query_to_execute); +} +void DatabaseMaterializeMySQL::dumpMySQLDatabase() +{ + mysqlxx::PoolWithFailover::Entry connection = pool.get(); + + connection->query("FLUSH TABLES;").execute(); + connection->query("FLUSH TABLES WITH READ LOCK;").execute(); + + MasterStatusInfo master_status = fetchMasterStatus(connection); + connection->query("SET SESSION TRANSACTION ISOLATION LEVEL REPEATABLE READ;").execute(); + connection->query("START TRANSACTION /*!40100 WITH CONSISTENT SNAPSHOT */;").execute(); + + std::vector tables_in_db = fetchTablesInDB(connection, mysql_database_name); + connection->query("UNLOCK TABLES;").execute(); + + for (const auto & dumping_table_name : tables_in_db) + { + String query_prefix = "/* Dumping " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(dumping_table_name) + " for " + + backQuoteIfNeed(database_name) + " Database */ "; + + tryToExecuteQuery(query_prefix + " DROP TABLE IF EXISTS " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(dumping_table_name)); + tryToExecuteQuery(query_prefix + getCreateQuery(connection, mysql_database_name, dumping_table_name)); + + Context context = global_context; + context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + context.setCurrentQueryId(""); // generate random query_id + BlockIO streams = executeQuery(query_prefix + " INSERT INTO " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(dumping_table_name), context, true); + + if (!streams.out) + throw Exception("LOGICAL ERROR out stream is undefined.", ErrorCodes::LOGICAL_ERROR); + + MySQLBlockInputStream input( + connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(dumping_table_name), + streams.out->getHeader(), DEFAULT_BLOCK_SIZE); + + copyData(input, *streams.out /*, is_quit*/); + /// TODO: 启动slave, 监听事件 + } +} +void DatabaseMaterializeMySQL::synchronization() +{ + setThreadName("MySQLDBSync"); + + try + { + std::unique_lock lock{sync_mutex}; + + /// Check database is exists in ClickHouse. + while (!sync_quit && !DatabaseCatalog::instance().isDatabaseExist(database_name)) + sync_cond.wait_for(lock, std::chrono::seconds(1)); + + /// 查找一下位点文件, 如果不存在需要清理目前的数据库, 然后dump全量数据. + dumpMySQLDatabase(); + } + catch(...) + { + tryLogCurrentException(log); + } +} } diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h index 136df735c79..2d00e4888fb 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB { @@ -29,6 +30,19 @@ private: String mysql_database_name; mutable mysqlxx::Pool pool; + + void synchronization(); + + void dumpMySQLDatabase(); + + void tryToExecuteQuery(const String & query_to_execute); + + String getCreateQuery(const mysqlxx::Pool::Entry & connection, const String & database, const String & table_name); + + mutable std::mutex sync_mutex; + std::atomic sync_quit{false}; + std::condition_variable sync_cond; + ThreadFromGlobalPool thread{&DatabaseMaterializeMySQL::synchronization, this}; }; } From 86c6be4cd961a00ccff5fb05d7c04436acf6d644 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 12 May 2020 17:38:00 +0800 Subject: [PATCH 064/374] ISSUES-4006 fix create query convert --- src/Databases/DatabaseFactory.cpp | 2 +- .../MySQL/DatabaseMaterializeMySQL.cpp | 15 +- src/Databases/MySQL/MasterStatusInfo.cpp | 6 + src/Databases/MySQL/MasterStatusInfo.h | 3 - .../MySQL/CreateQueryConvertVisitor.cpp | 282 ++++++++++++++---- .../MySQL/CreateQueryConvertVisitor.h | 37 ++- .../gtest_create_query_convert_visitor.cpp | 218 +++++++------- src/Parsers/MySQL/ASTDeclareIndex.cpp | 4 +- src/Parsers/MySQL/ASTDeclareOption.cpp | 8 +- src/Parsers/MySQL/ASTDeclareTableOptions.cpp | 7 +- .../MySQL/tests/gtest_create_parser.cpp | 9 + 11 files changed, 403 insertions(+), 188 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index cf7e93a0e1f..abfbfbd8878 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -81,7 +81,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String throw Exception("Database engine " + engine_name + " cannot have arguments", ErrorCodes::BAD_ARGUMENTS); if (engine_define->engine->parameters || engine_define->partition_by || engine_define->primary_key || engine_define->order_by || - engine_define->sample_by || engine_define->settings) + engine_define->sample_by || (engine_name != "MySQL" && engine_define->settings)) throw Exception("Database engine " + engine_name + " cannot have parameters, primary_key, order_by, sample_by, settings", ErrorCodes::UNKNOWN_ELEMENT_IN_AST); diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 288e99cd906..178fca8bbe4 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -88,7 +88,7 @@ String DatabaseMaterializeMySQL::getCreateQuery(const mysqlxx::Pool::Entry & con { Block show_create_table_header{ {std::make_shared(), "Table"}, - {std::make_shared(), "Create Table"}, + {std::make_shared(), "Create Table"}, }; MySQLBlockInputStream show_create_table( @@ -104,8 +104,12 @@ String DatabaseMaterializeMySQL::getCreateQuery(const mysqlxx::Pool::Entry & con MySQLParser::ParserCreateQuery p_create_query; ASTPtr ast = parseQuery(p_create_query, create_query.data, create_query.data + create_query.size, "", 0, 0); + if (!ast || !ast->as()) + throw Exception("LOGICAL ERROR: ast cannot cast to MySQLParser::ASTCreateQuery.", ErrorCodes::LOGICAL_ERROR); + WriteBufferFromOwnString out; - MySQLVisitor::CreateQueryConvertVisitor::Data data{.out = out}; + ast->as()->database = database; + MySQLVisitor::CreateQueryConvertVisitor::Data data{.out = out, .context = global_context}; MySQLVisitor::CreateQueryConvertVisitor visitor(data); visitor.visit(ast); return out.str(); @@ -152,12 +156,12 @@ void DatabaseMaterializeMySQL::dumpMySQLDatabase() + backQuoteIfNeed(database_name) + " Database */ "; tryToExecuteQuery(query_prefix + " DROP TABLE IF EXISTS " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(dumping_table_name)); - tryToExecuteQuery(query_prefix + getCreateQuery(connection, mysql_database_name, dumping_table_name)); + tryToExecuteQuery(query_prefix + getCreateQuery(connection, database_name, dumping_table_name)); Context context = global_context; context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; context.setCurrentQueryId(""); // generate random query_id - BlockIO streams = executeQuery(query_prefix + " INSERT INTO " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(dumping_table_name), context, true); + BlockIO streams = executeQuery( query_prefix + " INSERT INTO " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(dumping_table_name) + " VALUES", context, true); if (!streams.out) throw Exception("LOGICAL ERROR out stream is undefined.", ErrorCodes::LOGICAL_ERROR); @@ -178,10 +182,11 @@ void DatabaseMaterializeMySQL::synchronization() { std::unique_lock lock{sync_mutex}; - /// Check database is exists in ClickHouse. + LOG_DEBUG(log, "Checking " + database_name + " database status."); while (!sync_quit && !DatabaseCatalog::instance().isDatabaseExist(database_name)) sync_cond.wait_for(lock, std::chrono::seconds(1)); + LOG_DEBUG(log, database_name + " database status is OK."); /// 查找一下位点文件, 如果不存在需要清理目前的数据库, 然后dump全量数据. dumpMySQLDatabase(); } diff --git a/src/Databases/MySQL/MasterStatusInfo.cpp b/src/Databases/MySQL/MasterStatusInfo.cpp index 5c53db22bc4..007f611520f 100644 --- a/src/Databases/MySQL/MasterStatusInfo.cpp +++ b/src/Databases/MySQL/MasterStatusInfo.cpp @@ -2,5 +2,11 @@ namespace DB { +MasterStatusInfo::MasterStatusInfo( + String binlog_file_, UInt64 binlog_position_, String binlog_do_db_, String binlog_ignore_db_, String executed_gtid_set_) + : binlog_file(binlog_file_), binlog_position(binlog_position_), binlog_do_db(binlog_do_db_), binlog_ignore_db(binlog_ignore_db_), + executed_gtid_set(executed_gtid_set_) +{ +} } diff --git a/src/Databases/MySQL/MasterStatusInfo.h b/src/Databases/MySQL/MasterStatusInfo.h index aea50aa3bdd..ba953ceb1dc 100644 --- a/src/Databases/MySQL/MasterStatusInfo.h +++ b/src/Databases/MySQL/MasterStatusInfo.h @@ -20,8 +20,5 @@ struct MasterStatusInfo }; - -std::shared_ptr fetchMasterStatusInfo(mysqlxx::Connection * connection); - } diff --git a/src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp b/src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp index 9c2fe24d48b..c52588cab64 100644 --- a/src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp +++ b/src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp @@ -1,14 +1,20 @@ #include #include #include -#include #include #include #include #include +#include #include #include +#include +#include +#include +#include + + namespace DB { @@ -22,42 +28,7 @@ namespace ErrorCodes namespace MySQLVisitor { -void CreateQueryMatcher::visit(ASTPtr & ast, Data & data) -{ - if (auto * t = ast->as()) - visit(*t, ast, data); -} -void CreateQueryMatcher::visit(ASTCreateQuery & create, ASTPtr & /*ast*/, Data & data) -{ - if (create.like_table) - throw Exception("Cannot convert create like statement to ClickHouse SQL", ErrorCodes::NOT_IMPLEMENTED); - - data.out << "CREATE TABLE " << (create.if_not_exists ? "IF NOT EXISTS" : "") - << (create.database.empty() ? "" : backQuoteIfNeed(create.database) + ".") << backQuoteIfNeed(create.table) << "("; - - if (create.columns_list) - visitColumns(*create.columns_list->as(), create.columns_list, data); - - data.out << ") ENGINE = MergeTree()"; -} - -void CreateQueryMatcher::visitColumns(ASTCreateDefines & create_defines, ASTPtr & /*ast*/, Data & data) -{ - if (!create_defines.columns || create_defines.columns->children.empty()) - throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); - - bool is_first = true; - for (auto & column : create_defines.columns->children) - { - if (!is_first) - data.out << ","; - - is_first = false; - visitColumns(*column->as(), column, data); - } -} - -static String convertDataType(const String & type_name, const ASTPtr & arguments, bool is_unsigned, bool /*is_national*/) +static String convertDataType(const String & type_name, const ASTPtr & arguments, bool is_unsigned) { if (type_name == "TINYINT") return is_unsigned ? "UInt8" : "Int8"; @@ -74,7 +45,17 @@ static String convertDataType(const String & type_name, const ASTPtr & arguments else if (type_name == "DOUBLE" || type_name == "PRECISION" || type_name == "REAL") return "Float64"; else if (type_name == "DECIMAL" || type_name == "DEC" || type_name == "NUMERIC" || type_name == "FIXED") - return arguments ? "Decimal(" + queryToString(arguments) + ")" : "Decimal(10, 0)"; + { + if (!arguments) + return "Decimal(10, 0)"; + else if (arguments->children.size() == 1) + return "Decimal(" + queryToString(arguments) + ", 0)"; + else if (arguments->children.size() == 2) + return "Decimal(" + queryToString(arguments) + ")"; + else + throw Exception("Decimal data type family must have exactly two arguments: precision and scale", ErrorCodes::UNKNOWN_TYPE); + } + if (type_name == "DATE") return "Date"; @@ -91,25 +72,74 @@ static String convertDataType(const String & type_name, const ASTPtr & arguments return "String"; } -void CreateQueryMatcher::visitColumns(const ASTDeclareColumn & declare_column, ASTPtr &, Data & data) +static String convertDataType(const String & type_name, const ASTPtr & arguments, bool is_unsigned, bool is_nullable) { - data.out << declare_column.name << " "; + return (is_nullable ? "Nullable(" : "") + convertDataType(type_name, arguments, is_unsigned) + (is_nullable ? ")" : ""); +} +void CreateQueryMatcher::visit(ASTPtr & ast, Data & data) +{ + if (auto * t = ast->as()) + visit(*t, ast, data); +} + +void CreateQueryMatcher::visit(MySQLParser::ASTCreateQuery & create, ASTPtr &, Data & data) +{ + if (create.like_table) + throw Exception("Cannot convert create like statement to ClickHouse SQL", ErrorCodes::NOT_IMPLEMENTED); + + if (create.columns_list) + visit(*create.columns_list->as(), create.columns_list, data); + + if (create.partition_options) + visit(*create.partition_options->as(), create.partition_options, data); + + auto expression_list = std::make_shared(); + expression_list->children = data.primary_keys; + + data.out << "CREATE TABLE " << (create.if_not_exists ? "IF NOT EXISTS" : "") + << (create.database.empty() ? "" : backQuoteIfNeed(create.database) + ".") << backQuoteIfNeed(create.table) + << "(" << queryToString(InterpreterCreateQuery::formatColumns(data.columns_name_and_type)) << ") ENGINE = MergeTree()" + " PARTITION BY " << queryToString(data.getFormattedPartitionByExpression()) + << " ORDER BY " << queryToString(data.getFormattedOrderByExpression()); +} + +void CreateQueryMatcher::visit(MySQLParser::ASTCreateDefines & create_defines, ASTPtr &, Data & data) +{ + if (!create_defines.columns || create_defines.columns->children.empty()) + throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); + + if (create_defines.indices) + { + for (auto & index : create_defines.indices->children) + visit(*index->as(), index, data); + } + + for (auto & column : create_defines.columns->children) + visit(*column->as(), column, data); +} + +void CreateQueryMatcher::visit(const MySQLParser::ASTDeclareIndex & declare_index, ASTPtr &, Data & data) +{ + if (startsWith(declare_index.index_type, "PRIMARY_KEY_")) + data.addPrimaryKey(declare_index.index_columns); +} + +void CreateQueryMatcher::visit(const MySQLParser::ASTDeclareColumn & declare_column, ASTPtr &, Data & data) +{ if (!declare_column.data_type) throw Exception("Missing type in definition of column.", ErrorCodes::UNKNOWN_TYPE); - bool is_unsigned = false; - bool is_nullable = true; - bool is_national = false; + bool is_nullable = true, is_unsigned = false; if (declare_column.column_options) { - if (ASTDeclareOptions * options = declare_column.column_options->as()) + if (MySQLParser::ASTDeclareOptions * options = declare_column.column_options->as()) { if (options->changes.count("is_null")) is_nullable = options->changes["is_null"]->as()->value.safeGet(); - if (options->changes.count("is_national")) - is_national = options->changes["is_national"]->as()->value.safeGet(); + if (options->changes.count("primary_key")) + data.addPrimaryKey(std::make_shared(declare_column.name)); if (options->changes.count("is_unsigned")) is_unsigned = options->changes["is_unsigned"]->as()->value.safeGet(); @@ -119,13 +149,161 @@ void CreateQueryMatcher::visitColumns(const ASTDeclareColumn & declare_column, A } if (ASTFunction * function = declare_column.data_type->as()) - data.out << (is_nullable ? "Nullable(" : "") - << convertDataType(Poco::toUpper(function->name), function->arguments, is_unsigned, is_national) - << (is_nullable ? ")" : ""); + data.columns_name_and_type.emplace_back(declare_column.name, + DataTypeFactory::instance().get(convertDataType(Poco::toUpper(function->name), function->arguments, is_unsigned, is_nullable))); + else if (ASTIdentifier * identifier = declare_column.data_type->as()) + data.columns_name_and_type.emplace_back(declare_column.name, + DataTypeFactory::instance().get(convertDataType(Poco::toUpper(identifier->name), ASTPtr{}, is_unsigned, is_nullable))); + else + throw Exception("Unsupported MySQL data type " + queryToString(declare_column.data_type) + ".", ErrorCodes::NOT_IMPLEMENTED); +} - if (ASTIdentifier * identifier = declare_column.data_type->as()) - data.out << (is_nullable ? "Nullable(" : "") << convertDataType(Poco::toUpper(identifier->name), ASTPtr{}, is_unsigned, is_national) - << (is_nullable ? ")" : ""); +void CreateQueryMatcher::visit(const MySQLParser::ASTDeclarePartitionOptions & declare_partition_options, ASTPtr &, Data & data) +{ + data.addPartitionKey(declare_partition_options.partition_expression); +} + +void CreateQueryMatcher::Data::addPrimaryKey(const ASTPtr & primary_key) +{ + if (const auto & function_index_columns = primary_key->as()) + { + if (function_index_columns->name != "tuple") + throw Exception("Unable to parse function primary key from MySQL.", ErrorCodes::NOT_IMPLEMENTED); + + for (const auto & index_column : function_index_columns->arguments->children) + primary_keys.emplace_back(index_column); + } + else if (const auto & expression_index_columns = primary_key->as()) + { + for (const auto & index_column : expression_index_columns->children) + primary_keys.emplace_back(index_column); + } + else + primary_keys.emplace_back(primary_key); +} + +void CreateQueryMatcher::Data::addPartitionKey(const ASTPtr & partition_key) +{ + if (const auto & function_partition_columns = partition_key->as()) + { + if (function_partition_columns->name != "tuple") + throw Exception("Unable to parse function partition by from MySQL.", ErrorCodes::NOT_IMPLEMENTED); + + for (const auto & partition_column : function_partition_columns->arguments->children) + partition_keys.emplace_back(partition_column); + } + else if (const auto & expression_partition_columns = partition_key->as()) + { + for (const auto & partition_column : expression_partition_columns->children) + partition_keys.emplace_back(partition_column); + } + else + partition_keys.emplace_back(partition_key); +} + +ASTPtr CreateQueryMatcher::Data::getFormattedOrderByExpression() +{ + if (primary_keys.empty()) + return makeASTFunction("tuple"); + + /// TODO: support unique key & key + const auto function = std::make_shared(); + function->name = "tuple"; + function->arguments = std::make_shared(); + function->children.push_back(function->arguments); + function->arguments->children = primary_keys; + + return function; +} + +template +Field choiceBetterRangeSize(TType min, TType max, size_t max_ranges, size_t min_size_pre_range) +{ + UInt64 interval = UInt64(max) - min; + size_t calc_rows_pre_range = std::ceil(interval / double(max_ranges)); + size_t rows_pre_range = std::max(min_size_pre_range, calc_rows_pre_range); + + if (rows_pre_range >= interval) + return Null(); + + return rows_pre_range > std::numeric_limits::max() ? Field(UInt64(rows_pre_range)) : Field(TType(rows_pre_range)); +} + +ASTPtr CreateQueryMatcher::Data::getFormattedPartitionByExpression() +{ + ASTPtr partition_columns = std::make_shared(); + + if (!partition_keys.empty()) + partition_columns->children = partition_keys; + else if (!primary_keys.empty()) + { + ASTPtr expr_list = std::make_shared(); + expr_list->children = primary_keys; + + auto syntax = SyntaxAnalyzer(context).analyze(expr_list, columns_name_and_type); + auto index_expr = ExpressionAnalyzer(expr_list, syntax, context).getActions(false); + const NamesAndTypesList & required_names_and_types = index_expr->getRequiredColumnsWithTypes(); + + const auto & addPartitionColumn = [&](const String & column_name, const DataTypePtr & type, Field better_pre_range_size) + { + partition_columns->children.emplace_back(std::make_shared(column_name)); + + if (type->isNullable()) + partition_columns->children.back() = makeASTFunction("assumeNotNull", partition_columns->children.back()); + + if (!better_pre_range_size.isNull()) + partition_columns->children.back() + = makeASTFunction("divide", partition_columns->children.back(), std::make_shared(better_pre_range_size)); + }; + + for (const auto & required_name_and_type : required_names_and_types) + { + DataTypePtr assume_not_null = required_name_and_type.type; + if (assume_not_null->isNullable()) + assume_not_null = (static_cast(*assume_not_null)).getNestedType(); + + WhichDataType which(assume_not_null); + if (which.isInt8()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isInt16()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isInt32()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isInt64()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isUInt8()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isUInt16()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isUInt32()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isUInt64()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isDateOrDateTime()) + { + partition_columns->children.emplace_back(std::make_shared(required_name_and_type.name)); + + if (required_name_and_type.type->isNullable()) + partition_columns->children.back() = makeASTFunction("assumeNotNull", partition_columns->children.back()); + + partition_columns->children.back() = makeASTFunction("toYYYYMM", partition_columns->children.back()); + } + } + } + + const auto function = std::make_shared(); + function->name = "tuple"; + function->arguments = partition_columns; + function->children.push_back(function->arguments); + return function; } } diff --git a/src/Interpreters/MySQL/CreateQueryConvertVisitor.h b/src/Interpreters/MySQL/CreateQueryConvertVisitor.h index 2c0c9ad6d7f..c4c31f54b88 100644 --- a/src/Interpreters/MySQL/CreateQueryConvertVisitor.h +++ b/src/Interpreters/MySQL/CreateQueryConvertVisitor.h @@ -1,15 +1,16 @@ #pragma once +#include #include #include #include +#include #include +#include namespace DB { -using namespace MySQLParser; - namespace MySQLVisitor { @@ -23,18 +24,42 @@ public: { /// SETTINGS WriteBuffer & out; - std::string declare_columns; + const Context & context; + size_t max_ranges; + size_t min_rows_pre_range; + + ASTs primary_keys; + ASTs partition_keys; + NamesAndTypesList columns_name_and_type; + + void addPrimaryKey(const ASTPtr & primary_key); + + void addPartitionKey(const ASTPtr & partition_key); + + ASTPtr getFormattedOrderByExpression(); + + ASTPtr getFormattedPartitionByExpression(); }; static void visit(ASTPtr & ast, Data & data); static bool needChildVisit(ASTPtr &, const ASTPtr &) { return false; } private: - static void visit(ASTCreateQuery & create, ASTPtr & ast, Data &); + static void visit(MySQLParser::ASTCreateQuery & create, ASTPtr & ast, Data &); - static void visitColumns(ASTCreateDefines & create_defines, ASTPtr & ast, Data & data); + static void visit(MySQLParser::ASTCreateDefines & create_defines, ASTPtr & ast, Data & data); - static void visitColumns(const ASTDeclareColumn & declare_column, ASTPtr & ast, Data & data); + static void visit(const MySQLParser::ASTDeclareIndex & declare_index, ASTPtr & ast, Data & data); + + static void visit(const MySQLParser::ASTDeclareColumn & declare_column, ASTPtr & ast, Data & data); + + static void visit(const MySQLParser::ASTDeclarePartitionOptions & declare_partition_options, ASTPtr & ast, Data & data); + +// static void visitPartitionBy(MySQLParser::ASTCreateQuery & create, ASTPtr & ast, Data & data); + +// static void visitPartitionBy(MySQLParser::ASTCreateDefines & create_defines, ASTPtr & ast, Data & data); + +// static void visitPartitionBy(const MySQLParser::ASTDeclarePartitionOptions & partition_options, ASTPtr & ast, Data & data); // static void visitColumns(const ASTFunction & declare_column, ASTPtr & ast, Data & data); // static void visit(ASTTableJoin & join, const ASTPtr & ast, Data &); diff --git a/src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp b/src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp index 6a8ae26e4c4..beed50b1450 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp @@ -2,153 +2,149 @@ #include #include #include +#include #include using namespace DB; using namespace MySQLParser; using namespace MySQLVisitor; -TEST(CreateQueryConvert, SimpleNumbersType) +static ContextShared * contextShared() +{ + static SharedContextHolder shared = Context::createShared(); + return shared.get(); +} + +static String convert(const String & input) { ParserCreateQuery p_create_query; - String input = "CREATE TABLE test(a tinyint, b SMALLINT, c MEDIUMINT, d INT, e INTEGER, f BIGINT, g DECIMAL, h DEC, i NUMERIC, j " - "FIXED, k FLOAT, l DOUBLE, m DOUBLE PRECISION, n REAL)"; ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); WriteBufferFromOwnString out; - CreateQueryConvertVisitor::Data data{.out = out}; + CreateQueryConvertVisitor::Data data{ + .out = out, .context = Context::createGlobal(contextShared()), .max_ranges = 1000, .min_rows_pre_range = 1000000}; CreateQueryConvertVisitor visitor(data); visitor.visit(ast); + return out.str(); +} + +TEST(CreateQueryConvert, TestConvertNumberColumnsType) +{ + EXPECT_EQ( + convert("CREATE TABLE test(a tinyint, b SMALLINT, c MEDIUMINT, d INT, e INTEGER, f BIGINT, g DECIMAL, h DEC, i NUMERIC, j FIXED, k " + "FLOAT, l DOUBLE, m DOUBLE PRECISION, n REAL)"), + "CREATE TABLE test(`a` Nullable(Int8), `b` Nullable(Int16), `c` Nullable(Int32), `d` Nullable(Int32), `e` Nullable(Int32), `f` " + "Nullable(Int64), `g` Nullable(Decimal(10, 0)), `h` Nullable(Decimal(10, 0)), `i` Nullable(Decimal(10, 0)), `j` " + "Nullable(Decimal(10, 0)), `k` Nullable(Float32), `l` Nullable(Float64), `m` Nullable(Float64), `n` Nullable(Float64)) ENGINE = " + "MergeTree() PARTITION BY tuple() ORDER BY tuple()"); EXPECT_EQ( - out.str(), - "CREATE TABLE test(a Nullable(Int8),b Nullable(Int16),c Nullable(Int32),d Nullable(Int32),e Nullable(Int32),f Nullable(Int64),g " - "Nullable(Decimal(10, 0)),h Nullable(Decimal(10, 0)),i Nullable(Decimal(10, 0)),j Nullable(Decimal(10, 0)),k Nullable(Float32),l " - "Nullable(Float64),m Nullable(Float64),n Nullable(Float64)) ENGINE = MergeTree()"); + convert("CREATE TABLE test(a tinyint(1), b SMALLINT(1), c MEDIUMINT(1), d INT(1), e INTEGER(1), f BIGINT(1), g DECIMAL(1), h " + "DEC(2, 1), i NUMERIC(4, 3), j FIXED(6, 5), k FLOAT(1), l DOUBLE(1, 2), m DOUBLE PRECISION(3, 4), n REAL(5, 6))"), + "CREATE TABLE test(`a` Nullable(Int8), `b` Nullable(Int16), `c` Nullable(Int32), `d` Nullable(Int32), `e` Nullable(Int32), `f` " + "Nullable(Int64), `g` Nullable(Decimal(1, 0)), `h` Nullable(Decimal(2, 1)), `i` Nullable(Decimal(4, 3)), `j` Nullable(Decimal(6, " + "5)), `k` Nullable(Float32), `l` Nullable(Float64), `m` Nullable(Float64), `n` Nullable(Float64)) ENGINE = MergeTree() PARTITION " + "BY tuple() ORDER BY tuple()"); + + /// UNSIGNED + EXPECT_EQ( + convert("CREATE TABLE test(a tinyint UNSIGNED, b SMALLINT(1) UNSIGNED, c MEDIUMINT(1) UNSIGNED, d INT(1) UNSIGNED, e INTEGER(1), f " + "BIGINT(1) UNSIGNED, g DECIMAL(1) UNSIGNED, h DEC(2, 1) UNSIGNED, i NUMERIC(4, 3) UNSIGNED, j FIXED(6, 5) UNSIGNED, k FLOAT(1) " + "UNSIGNED, l DOUBLE(1, 2) UNSIGNED, m DOUBLE PRECISION(3, 4) UNSIGNED, n REAL(5, 6) UNSIGNED)"), + "CREATE TABLE test(`a` Nullable(UInt8), `b` Nullable(UInt16), `c` Nullable(UInt32), `d` Nullable(UInt32), `e` Nullable(Int32), `f` " + "Nullable(UInt64), `g` Nullable(Decimal(1, 0)), `h` Nullable(Decimal(2, 1)), `i` Nullable(Decimal(4, 3)), `j` Nullable(Decimal(6, " + "5)), `k` Nullable(Float32), `l` Nullable(Float64), `m` Nullable(Float64), `n` Nullable(Float64)) ENGINE = MergeTree() PARTITION " + "BY tuple() ORDER BY tuple()"); + + /// NOT NULL + EXPECT_EQ( + convert("CREATE TABLE test(a tinyint NOT NULL, b SMALLINT(1) NOT NULL, c MEDIUMINT(1) NOT NULL, d INT(1) NOT NULL, e INTEGER(1), f " + "BIGINT(1) NOT NULL, g DECIMAL(1) NOT NULL, h DEC(2, 1) NOT NULL, i NUMERIC(4, 3) NOT NULL, j FIXED(6, 5) NOT NULL, k FLOAT(1) NOT " + "NULL, l DOUBLE(1, 2) NOT NULL, m DOUBLE PRECISION(3, 4) NOT NULL, n REAL(5, 6) NOT NULL)"), + "CREATE TABLE test(`a` Int8, `b` Int16, `c` Int32, `d` Int32, `e` Nullable(Int32), `f` Int64, `g` Decimal(1, 0), `h` Decimal(2, " + "1), `i` Decimal(4, 3), `j` Decimal(6, 5), `k` Float32, `l` Float64, `m` Float64, `n` Float64) ENGINE = MergeTree() PARTITION BY " + "tuple() ORDER BY tuple()"); + + /// ZEROFILL + EXPECT_EQ( + convert("CREATE TABLE test(a tinyint ZEROFILL, b SMALLINT(1) ZEROFILL, c MEDIUMINT(1) ZEROFILL, d INT(1) ZEROFILL, e INTEGER(1), f " + "BIGINT(1) ZEROFILL, g DECIMAL(1) ZEROFILL, h DEC(2, 1) ZEROFILL, i NUMERIC(4, 3) ZEROFILL, j FIXED(6, 5) ZEROFILL, k FLOAT(1) " + "ZEROFILL, l DOUBLE(1, 2) ZEROFILL, m DOUBLE PRECISION(3, 4) ZEROFILL, n REAL(5, 6) ZEROFILL)"), + "CREATE TABLE test(`a` Nullable(UInt8), `b` Nullable(UInt16), `c` Nullable(UInt32), `d` Nullable(UInt32), `e` Nullable(Int32), `f` " + "Nullable(UInt64), `g` Nullable(Decimal(1, 0)), `h` Nullable(Decimal(2, 1)), `i` Nullable(Decimal(4, 3)), `j` Nullable(Decimal(6, " + "5)), `k` Nullable(Float32), `l` Nullable(Float64), `m` Nullable(Float64), `n` Nullable(Float64)) ENGINE = MergeTree() PARTITION " + "BY tuple() ORDER BY tuple()"); } -TEST(CreateQueryConvert, NumbersTypeWithLength) +TEST(CreateQueryConvert, TestConvertDateTimesColumnsType) { - ParserCreateQuery p_create_query; - String input = "CREATE TABLE test(a tinyint(1), b SMALLINT(1), c MEDIUMINT(1), d INT(1), e INTEGER(1), f BIGINT(1), g DECIMAL(1), h DEC(1, 2), i NUMERIC(3, 4), j " - "FIXED(5, 6), k FLOAT(1), l DOUBLE(1, 2), m DOUBLE PRECISION(3, 4), n REAL(5, 6))"; - ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); - - WriteBufferFromOwnString out; - CreateQueryConvertVisitor::Data data{.out = out}; - CreateQueryConvertVisitor visitor(data); - visitor.visit(ast); + EXPECT_EQ( + convert("CREATE TABLE test(a DATE, b DATETIME, c TIMESTAMP, d TIME, e year)"), + "CREATE TABLE test(`a` Nullable(Date), `b` Nullable(DateTime), `c` Nullable(DateTime), `d` Nullable(DateTime64(3)), `e` " + "Nullable(Int16)) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY tuple()"); EXPECT_EQ( - out.str(), - "CREATE TABLE test(a Nullable(Int8),b Nullable(Int16),c Nullable(Int32),d Nullable(Int32),e Nullable(Int32),f Nullable(Int64),g " - "Nullable(Decimal(1)),h Nullable(Decimal(1, 2)),i Nullable(Decimal(3, 4)),j Nullable(Decimal(5, 6)),k Nullable(Float32),l " - "Nullable(Float64),m Nullable(Float64),n Nullable(Float64)) ENGINE = MergeTree()"); -} - -TEST(CreateQueryConvert, NumbersTypeWithUnsigned) -{ - ParserCreateQuery p_create_query; - String input = "CREATE TABLE test(a tinyint UNSIGNED, b SMALLINT(1) UNSIGNED, c MEDIUMINT(1) UNSIGNED, d INT(1) UNSIGNED, e " - "INTEGER(1), f BIGINT(1) UNSIGNED, g DECIMAL(1) UNSIGNED, h DEC(1, 2) UNSIGNED, i NUMERIC(3, 4) UNSIGNED, j FIXED(5, 6) " - "UNSIGNED, k FLOAT(1) UNSIGNED, l DOUBLE(1, 2) UNSIGNED, m DOUBLE PRECISION(3, 4) UNSIGNED, n REAL(5, 6) UNSIGNED)"; - ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); - - WriteBufferFromOwnString out; - CreateQueryConvertVisitor::Data data{.out = out}; - CreateQueryConvertVisitor visitor(data); - visitor.visit(ast); + convert("CREATE TABLE test(a DATE, b DATETIME(1), c TIMESTAMP(1), d TIME(1), e year(4))"), + "CREATE TABLE test(`a` Nullable(Date), `b` Nullable(DateTime), `c` Nullable(DateTime), `d` Nullable(DateTime64(3)), `e` " + "Nullable(Int16)) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY tuple()"); EXPECT_EQ( - out.str(), - "CREATE TABLE test(a Nullable(UInt8),b Nullable(UInt16),c Nullable(UInt32),d Nullable(UInt32),e Nullable(Int32),f " - "Nullable(UInt64),g Nullable(Decimal(1)),h Nullable(Decimal(1, 2)),i Nullable(Decimal(3, 4)),j Nullable(Decimal(5, 6)),k " - "Nullable(Float32),l Nullable(Float64),m Nullable(Float64),n Nullable(Float64)) ENGINE = MergeTree()"); + convert( + "CREATE TABLE test(a DATE NOT NULL, b DATETIME(1) NOT NULL, c TIMESTAMP(1) NOT NULL, d TIME(1) NOT NULL, e year(4) NOT NULL)"), + "CREATE TABLE test(`a` Date, `b` DateTime, `c` DateTime, `d` DateTime64(3), `e` Int16) ENGINE = MergeTree() PARTITION BY tuple() " + "ORDER BY tuple()"); } -TEST(CreateQueryConvert, NumbersTypeWithNotNull) +TEST(CreateQueryConvert, TestConvertParitionOptions) { - ParserCreateQuery p_create_query; - String input = "CREATE TABLE test(a tinyint NOT NULL, b SMALLINT(1) NOT NULL, c MEDIUMINT(1) NOT NULL, d INT(1) NOT NULL, e " - "INTEGER(1), f BIGINT(1) NOT NULL, g DECIMAL(1) NOT NULL, h DEC(1, 2) NOT NULL, i NUMERIC(3, 4) NOT NULL, j FIXED(5, 6) " - "NOT NULL, k FLOAT(1) NOT NULL, l DOUBLE(1, 2) NOT NULL, m DOUBLE PRECISION(3, 4) NOT NULL, n REAL(5, 6) NOT NULL)"; - ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); - - WriteBufferFromOwnString out; - CreateQueryConvertVisitor::Data data{.out = out}; - CreateQueryConvertVisitor visitor(data); - visitor.visit(ast); + EXPECT_EQ( + convert("CREATE TABLE test(a DATE NOT NULL) PARTITION BY HASH a"), + "CREATE TABLE test(`a` Date) ENGINE = MergeTree() PARTITION BY tuple(a) ORDER BY tuple()"); EXPECT_EQ( - out.str(), - "CREATE TABLE test(a Int8,b Int16,c Int32,d Int32,e Nullable(Int32),f Int64,g Decimal(1),h Decimal(1, 2),i Decimal(3, 4),j " - "Decimal(5, 6),k Float32,l Float64,m Float64,n Float64) ENGINE = MergeTree()"); -} - -TEST(CreateQueryConvert, NumbersTypeWithZeroFill) -{ - ParserCreateQuery p_create_query; - String input = "CREATE TABLE test(a tinyint ZEROFILL, b SMALLINT(1) ZEROFILL, c MEDIUMINT(1) ZEROFILL, d INT(1) ZEROFILL, e " - "INTEGER(1), f BIGINT(1) ZEROFILL, g DECIMAL(1) ZEROFILL, h DEC(1, 2) ZEROFILL, i NUMERIC(3, 4) ZEROFILL, j FIXED(5, 6) " - "ZEROFILL, k FLOAT(1) ZEROFILL, l DOUBLE(1, 2) ZEROFILL, m DOUBLE PRECISION(3, 4) ZEROFILL, n REAL(5, 6) ZEROFILL)"; - ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); - - WriteBufferFromOwnString out; - CreateQueryConvertVisitor::Data data{.out = out}; - CreateQueryConvertVisitor visitor(data); - visitor.visit(ast); + convert("CREATE TABLE test(a DATE NOT NULL) PARTITION BY LINEAR HASH a"), + "CREATE TABLE test(`a` Date) ENGINE = MergeTree() PARTITION BY tuple(a) ORDER BY tuple()"); EXPECT_EQ( - out.str(), - "CREATE TABLE test(a Nullable(UInt8),b Nullable(UInt16),c Nullable(UInt32),d Nullable(UInt32),e Nullable(Int32),f " - "Nullable(UInt64),g Nullable(Decimal(1)),h Nullable(Decimal(1, 2)),i Nullable(Decimal(3, 4)),j Nullable(Decimal(5, 6)),k " - "Nullable(Float32),l Nullable(Float64),m Nullable(Float64),n Nullable(Float64)) ENGINE = MergeTree()"); -} - -TEST(CreateQueryConvert, SimpleDateTimesType) -{ - ParserCreateQuery p_create_query; - String input = "CREATE TABLE test(a DATE, b DATETIME, c TIMESTAMP, d TIME, e year)"; - ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); - - WriteBufferFromOwnString out; - CreateQueryConvertVisitor::Data data{.out = out}; - CreateQueryConvertVisitor visitor(data); - visitor.visit(ast); + convert("CREATE TABLE test(a DATE NOT NULL) PARTITION BY RANGE(a)"), + "CREATE TABLE test(`a` Date) ENGINE = MergeTree() PARTITION BY tuple(a) ORDER BY tuple()"); EXPECT_EQ( - out.str(), - "CREATE TABLE test(a Nullable(Date),b Nullable(DateTime),c Nullable(DateTime),d Nullable(DateTime64),e Nullable(Int16)) ENGINE = " - "MergeTree()"); -} - -TEST(CreateQueryConvert, DateTimeTypesWithLength) -{ - ParserCreateQuery p_create_query; - String input = "CREATE TABLE test(a DATE, b DATETIME(1), c TIMESTAMP(1), d TIME(1), e year(4))"; - ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); - - WriteBufferFromOwnString out; - CreateQueryConvertVisitor::Data data{.out = out}; - CreateQueryConvertVisitor visitor(data); - visitor.visit(ast); + convert("CREATE TABLE test(a DATE NOT NULL, b INT) PARTITION BY RANGE COLUMNS(a, b)"), + "CREATE TABLE test(`a` Date, `b` Nullable(Int32)) ENGINE = MergeTree() PARTITION BY (a, b) ORDER BY tuple()"); EXPECT_EQ( - out.str(), - "CREATE TABLE test(a Nullable(Date),b Nullable(DateTime),c Nullable(DateTime),d Nullable(DateTime64),e Nullable(Int16)) ENGINE = " - "MergeTree()"); + convert("CREATE TABLE test(a DATE NOT NULL) PARTITION BY LIST(a)"), + "CREATE TABLE test(`a` Date) ENGINE = MergeTree() PARTITION BY tuple(a) ORDER BY tuple()"); + + EXPECT_EQ( + convert("CREATE TABLE test(a DATE NOT NULL, b INT) PARTITION BY LIST COLUMNS(a, b)"), + "CREATE TABLE test(`a` Date, `b` Nullable(Int32)) ENGINE = MergeTree() PARTITION BY (a, b) ORDER BY tuple()"); } -TEST(CreateQueryConvert, DateTimeTypesWithNotNull) +TEST(CreateQueryConvert, TestConvertPrimaryToPartitionBy) { - ParserCreateQuery p_create_query; - String input = "CREATE TABLE test(a DATE NOT NULL, b DATETIME(1) NOT NULL, c TIMESTAMP(1) NOT NULL, d TIME(1) NOT NULL, e year(4) NOT NULL)"; - ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); + EXPECT_EQ(convert("CREATE TABLE test(a DATE NOT NULL PRIMARY KEY)"), + "CREATE TABLE test(`a` Date) ENGINE = MergeTree() PARTITION BY tuple(toYYYYMM(a)) ORDER BY tuple(a)"); - WriteBufferFromOwnString out; - CreateQueryConvertVisitor::Data data{.out = out}; - CreateQueryConvertVisitor visitor(data); - visitor.visit(ast); + EXPECT_EQ(convert("CREATE TABLE test(a DATETIME NOT NULL PRIMARY KEY)"), + "CREATE TABLE test(`a` DateTime) ENGINE = MergeTree() PARTITION BY tuple(toYYYYMM(a)) ORDER BY tuple(a)"); - EXPECT_EQ(out.str(), "CREATE TABLE test(a Date,b DateTime,c DateTime,d DateTime64,e Int16) ENGINE = MergeTree()"); + EXPECT_EQ(convert("CREATE TABLE test(a TINYINT NOT NULL PRIMARY KEY)"), + "CREATE TABLE test(`a` Int8) ENGINE = MergeTree() PARTITION BY tuple(a) ORDER BY tuple(a)"); + + EXPECT_EQ(convert("CREATE TABLE test(a SMALLINT NOT NULL PRIMARY KEY)"), + "CREATE TABLE test(`a` Int16) ENGINE = MergeTree() PARTITION BY tuple(a) ORDER BY tuple(a)"); + + EXPECT_EQ(convert("CREATE TABLE test(a INT NOT NULL PRIMARY KEY)"), + "CREATE TABLE test(`a` Int32) ENGINE = MergeTree() PARTITION BY tuple(a / 4294968) ORDER BY tuple(a)"); + + EXPECT_EQ(convert("CREATE TABLE test(a BIGINT NOT NULL PRIMARY KEY)"), + "CREATE TABLE test(`a` Int64) ENGINE = MergeTree() PARTITION BY tuple(a / 18446744073709552) ORDER BY tuple(a)"); + + EXPECT_EQ( + convert("CREATE TABLE test(a BIGINT PRIMARY KEY)"), + "CREATE TABLE test(`a` Nullable(Int64)) ENGINE = MergeTree() PARTITION BY tuple(assumeNotNull(a) / 18446744073709552) ORDER BY " + "tuple(a)"); } + diff --git a/src/Parsers/MySQL/ASTDeclareIndex.cpp b/src/Parsers/MySQL/ASTDeclareIndex.cpp index 592e01e23a3..b48656efd71 100644 --- a/src/Parsers/MySQL/ASTDeclareIndex.cpp +++ b/src/Parsers/MySQL/ASTDeclareIndex.cpp @@ -207,7 +207,7 @@ bool ParserDeclareIndex::parseDeclareConstraintIndex(IParser::Pos & pos, String if (!p_identifier.parse(pos, temp_node, expected)) return false; - index_type = temp_node->as()->name; + index_type = "UNIQUE_" + temp_node->as()->name; } } else if (ParserKeyword("PRIMARY KEY").ignore(pos, expected)) @@ -218,7 +218,7 @@ bool ParserDeclareIndex::parseDeclareConstraintIndex(IParser::Pos & pos, String if (!p_identifier.parse(pos, temp_node, expected)) return false; - index_type = temp_node->as()->name; + index_type = "PRIMARY_KEY_" + temp_node->as()->name; } } else if (ParserKeyword("FOREIGN KEY").ignore(pos, expected)) diff --git a/src/Parsers/MySQL/ASTDeclareOption.cpp b/src/Parsers/MySQL/ASTDeclareOption.cpp index 9612d49cb12..188190e0a4f 100644 --- a/src/Parsers/MySQL/ASTDeclareOption.cpp +++ b/src/Parsers/MySQL/ASTDeclareOption.cpp @@ -122,12 +122,10 @@ bool ParserCharsetName::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &) while (true) { - if (isWhitespaceASCII(*pos->end)) - break; - else - { + if (!isWhitespaceASCII(*pos->end) && pos->type != TokenType::EndOfStream) ++pos; - } + else + break; } node = std::make_shared(String(begin, pos->end)); diff --git a/src/Parsers/MySQL/ASTDeclareTableOptions.cpp b/src/Parsers/MySQL/ASTDeclareTableOptions.cpp index 066e8c4c414..6d318a20f23 100644 --- a/src/Parsers/MySQL/ASTDeclareTableOptions.cpp +++ b/src/Parsers/MySQL/ASTDeclareTableOptions.cpp @@ -1,11 +1,10 @@ #include #include -#include -#include #include -#include +#include #include +#include #include namespace DB @@ -68,6 +67,8 @@ bool ParserDeclareTableOptions::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp { OptionDescribe("AUTO_INCREMENT", "auto_increment", std::make_shared()), OptionDescribe("AVG_ROW_LENGTH", "avg_row_length", std::make_shared()), + OptionDescribe("CHARSET", "character_set", std::make_shared()), + OptionDescribe("DEFAULT CHARSET", "character_set", std::make_shared()), OptionDescribe("CHARACTER SET", "character_set", std::make_shared()), OptionDescribe("DEFAULT CHARACTER SET", "character_set", std::make_shared()), OptionDescribe("CHECKSUM", "checksum", std::make_shared>()), diff --git a/src/Parsers/MySQL/tests/gtest_create_parser.cpp b/src/Parsers/MySQL/tests/gtest_create_parser.cpp index 5f752c29a7d..1b30a36d84a 100644 --- a/src/Parsers/MySQL/tests/gtest_create_parser.cpp +++ b/src/Parsers/MySQL/tests/gtest_create_parser.cpp @@ -31,3 +31,12 @@ TEST(CreateTableParser, SimpleCreate) EXPECT_EQ(ast->as()->table_options->as()->changes["engine"]->as()->name, "INNODB"); EXPECT_TRUE(ast->as()->partition_options->as()); } + +TEST(CreateTableParser, SS) +{ + ParserCreateQuery p_create_query; + String input = "CREATE TABLE `test_table_1` (`a` int DEFAULT NULL, `b` int DEFAULT NULL) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci"; + ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); + ast->dumpTree(std::cerr); + +} From 7203c6e1868299ed6081fee93c27b7bec7faa93a Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 12 May 2020 20:28:46 +0800 Subject: [PATCH 065/374] ISSUES-4006 unlock table on exception --- .../MySQL/DatabaseMaterializeMySQL.cpp | 125 ++++++------------ .../MySQL/DatabaseMaterializeMySQL.h | 4 +- src/Databases/MySQL/MasterStatusInfo.cpp | 74 ++++++++++- src/Databases/MySQL/MasterStatusInfo.h | 8 +- .../MySQL/CreateQueryConvertVisitor.cpp | 20 +-- .../MySQL/CreateQueryConvertVisitor.h | 22 +-- 6 files changed, 140 insertions(+), 113 deletions(-) diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 178fca8bbe4..508cdd5a861 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -18,6 +18,7 @@ # include # include # include +# include # include # include @@ -29,51 +30,6 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; } -static MasterStatusInfo fetchMasterStatus(const mysqlxx::PoolWithFailover::Entry & connection) -{ - Block header - { - {std::make_shared(), "File"}, - {std::make_shared(), "Position"}, - {std::make_shared(), "Binlog_Do_DB"}, - {std::make_shared(), "Binlog_Ignore_DB"}, - {std::make_shared(), "Executed_Gtid_Set"}, - }; - - MySQLBlockInputStream input(connection, "SHOW MASTER STATUS;", header, DEFAULT_BLOCK_SIZE); - Block master_status = input.read(); - - if (!master_status || master_status.rows() != 1) - throw Exception("Unable to get master status from MySQL.", ErrorCodes::LOGICAL_ERROR); - - return MasterStatusInfo - { - (*master_status.getByPosition(0).column)[0].safeGet(), - (*master_status.getByPosition(1).column)[0].safeGet(), - (*master_status.getByPosition(2).column)[0].safeGet(), - (*master_status.getByPosition(3).column)[0].safeGet(), - (*master_status.getByPosition(4).column)[0].safeGet() - }; -} - -static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entry & connection, const std::string & database) -{ - Block header{{std::make_shared(), "table_name"}}; - String query = "SELECT TABLE_NAME AS table_name FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA = " + quoteString(database); - - std::vector tables_in_db; - MySQLBlockInputStream input(connection, query, header, DEFAULT_BLOCK_SIZE); - - while (Block block = input.read()) - { - tables_in_db.reserve(tables_in_db.size() + block.rows()); - for (size_t index = 0; index < block.rows(); ++index) - tables_in_db.emplace_back((*block.getByPosition(0).column)[index].safeGet()); - } - - return tables_in_db; -} - DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( const Context & context, const String & database_name_, const String & metadata_path_, const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_) @@ -84,6 +40,28 @@ DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( /// TODO: 做简单的check, 失败即报错 } +void DatabaseMaterializeMySQL::tryToExecuteQuery(const String & query_to_execute) +{ + ReadBufferFromString istr(query_to_execute); + String dummy_string; + WriteBufferFromString ostr(dummy_string); + + try + { + Context context = global_context; + context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + context.setCurrentQueryId(""); // generate random query_id + executeQuery(istr, ostr, false, context, {}); + } + catch (...) + { + tryLogCurrentException(log, "Query " + query_to_execute + " wasn't finished successfully"); + throw; + } + + LOG_DEBUG(log, "Executed query: " << query_to_execute); +} + String DatabaseMaterializeMySQL::getCreateQuery(const mysqlxx::Pool::Entry & connection, const String & database, const String & table_name) { Block show_create_table_header{ @@ -115,53 +93,28 @@ String DatabaseMaterializeMySQL::getCreateQuery(const mysqlxx::Pool::Entry & con return out.str(); } -void DatabaseMaterializeMySQL::tryToExecuteQuery(const String & query_to_execute) -{ - ReadBufferFromString istr(query_to_execute); - String dummy_string; - WriteBufferFromString ostr(dummy_string); - - try - { - Context context = global_context; - context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - context.setCurrentQueryId(""); // generate random query_id - executeQuery(istr, ostr, false, context, {}); - } - catch (...) - { - tryLogCurrentException(log, "Query " + query_to_execute + " wasn't finished successfully"); - throw; - } - - LOG_DEBUG(log, "Executed query: " << query_to_execute); -} -void DatabaseMaterializeMySQL::dumpMySQLDatabase() +void DatabaseMaterializeMySQL::dumpMySQLDatabase(const std::function & is_cancelled) { mysqlxx::PoolWithFailover::Entry connection = pool.get(); - connection->query("FLUSH TABLES;").execute(); - connection->query("FLUSH TABLES WITH READ LOCK;").execute(); + MasterStatusInfo info(connection, mysql_database_name); - MasterStatusInfo master_status = fetchMasterStatus(connection); - connection->query("SET SESSION TRANSACTION ISOLATION LEVEL REPEATABLE READ;").execute(); - connection->query("START TRANSACTION /*!40100 WITH CONSISTENT SNAPSHOT */;").execute(); - - std::vector tables_in_db = fetchTablesInDB(connection, mysql_database_name); - connection->query("UNLOCK TABLES;").execute(); - - for (const auto & dumping_table_name : tables_in_db) + for (const auto & dumping_table_name : info.need_dumping_tables) { + if (is_cancelled()) + return; + + const auto & table_name = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(dumping_table_name); String query_prefix = "/* Dumping " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(dumping_table_name) + " for " + backQuoteIfNeed(database_name) + " Database */ "; - tryToExecuteQuery(query_prefix + " DROP TABLE IF EXISTS " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(dumping_table_name)); + tryToExecuteQuery(query_prefix + " DROP TABLE IF EXISTS " + table_name); tryToExecuteQuery(query_prefix + getCreateQuery(connection, database_name, dumping_table_name)); Context context = global_context; - context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; context.setCurrentQueryId(""); // generate random query_id - BlockIO streams = executeQuery( query_prefix + " INSERT INTO " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(dumping_table_name) + " VALUES", context, true); + context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + BlockIO streams = executeQuery(query_prefix + " INSERT INTO " + table_name + " VALUES", context, true); if (!streams.out) throw Exception("LOGICAL ERROR out stream is undefined.", ErrorCodes::LOGICAL_ERROR); @@ -170,7 +123,7 @@ void DatabaseMaterializeMySQL::dumpMySQLDatabase() connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(dumping_table_name), streams.out->getHeader(), DEFAULT_BLOCK_SIZE); - copyData(input, *streams.out /*, is_quit*/); + copyData(input, *streams.out, is_cancelled); /// TODO: 启动slave, 监听事件 } } @@ -187,8 +140,14 @@ void DatabaseMaterializeMySQL::synchronization() sync_cond.wait_for(lock, std::chrono::seconds(1)); LOG_DEBUG(log, database_name + " database status is OK."); - /// 查找一下位点文件, 如果不存在需要清理目前的数据库, 然后dump全量数据. - dumpMySQLDatabase(); + + Poco::File dumped_flag(getMetadataPath() + "/dumped.flag"); + + if (!dumped_flag.exists()) + { + dumpMySQLDatabase([&]() { return sync_quit.load(std::memory_order_seq_cst); }); + dumped_flag.createFile(); + } } catch(...) { diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h index 2d00e4888fb..b4fabdb20eb 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -33,10 +33,10 @@ private: void synchronization(); - void dumpMySQLDatabase(); - void tryToExecuteQuery(const String & query_to_execute); + void dumpMySQLDatabase(const std::function & is_cancelled); + String getCreateQuery(const mysqlxx::Pool::Entry & connection, const String & database, const String & table_name); mutable std::mutex sync_mutex; diff --git a/src/Databases/MySQL/MasterStatusInfo.cpp b/src/Databases/MySQL/MasterStatusInfo.cpp index 007f611520f..3a1a6c3d7dd 100644 --- a/src/Databases/MySQL/MasterStatusInfo.cpp +++ b/src/Databases/MySQL/MasterStatusInfo.cpp @@ -1,12 +1,84 @@ +#include +#include +#include #include +#include +#include namespace DB { -MasterStatusInfo::MasterStatusInfo( +/*MasterStatusInfo::MasterStatusInfo( String binlog_file_, UInt64 binlog_position_, String binlog_do_db_, String binlog_ignore_db_, String executed_gtid_set_) : binlog_file(binlog_file_), binlog_position(binlog_position_), binlog_do_db(binlog_do_db_), binlog_ignore_db(binlog_ignore_db_), executed_gtid_set(executed_gtid_set_) { +}*/ + +static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entry & connection, const std::string & database) +{ + Block header{{std::make_shared(), "table_name"}}; + String query = "SELECT TABLE_NAME AS table_name FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA = " + quoteString(database); + + std::vector tables_in_db; + MySQLBlockInputStream input(connection, query, header, DEFAULT_BLOCK_SIZE); + + while (Block block = input.read()) + { + tables_in_db.reserve(tables_in_db.size() + block.rows()); + for (size_t index = 0; index < block.rows(); ++index) + tables_in_db.emplace_back((*block.getByPosition(0).column)[index].safeGet()); + } + + return tables_in_db; +} + +MasterStatusInfo::MasterStatusInfo(mysqlxx::PoolWithFailover::Entry & connection, const String & database) +{ + bool locked_tables = false; + + try + { + connection->query("FLUSH TABLES;").execute(); + connection->query("FLUSH TABLES WITH READ LOCK;").execute(); + + locked_tables = true; + fetchMasterStatus(connection); + connection->query("SET SESSION TRANSACTION ISOLATION LEVEL REPEATABLE READ;").execute(); + connection->query("START TRANSACTION /*!40100 WITH CONSISTENT SNAPSHOT */;").execute(); + + need_dumping_tables = fetchTablesInDB(connection, database); + connection->query("UNLOCK TABLES;").execute(); + } + catch (...) + { + if (locked_tables) + connection->query("UNLOCK TABLES;").execute(); + + throw; + } +} +void MasterStatusInfo::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & connection) +{ + Block header + { + {std::make_shared(), "File"}, + {std::make_shared(), "Position"}, + {std::make_shared(), "Binlog_Do_DB"}, + {std::make_shared(), "Binlog_Ignore_DB"}, + {std::make_shared(), "Executed_Gtid_Set"}, + }; + + MySQLBlockInputStream input(connection, "SHOW MASTER STATUS;", header, DEFAULT_BLOCK_SIZE); + Block master_status = input.read(); + + if (!master_status || master_status.rows() != 1) + throw Exception("Unable to get master status from MySQL.", ErrorCodes::LOGICAL_ERROR); + + binlog_file = (*master_status.getByPosition(0).column)[0].safeGet(); + binlog_position = (*master_status.getByPosition(1).column)[0].safeGet(); + binlog_do_db = (*master_status.getByPosition(2).column)[0].safeGet(); + binlog_ignore_db = (*master_status.getByPosition(3).column)[0].safeGet(); + executed_gtid_set = (*master_status.getByPosition(4).column)[0].safeGet(); } } diff --git a/src/Databases/MySQL/MasterStatusInfo.h b/src/Databases/MySQL/MasterStatusInfo.h index ba953ceb1dc..ebc2a8c8d12 100644 --- a/src/Databases/MySQL/MasterStatusInfo.h +++ b/src/Databases/MySQL/MasterStatusInfo.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -14,8 +15,11 @@ struct MasterStatusInfo String binlog_ignore_db; String executed_gtid_set; - MasterStatusInfo( - String binlog_file_, UInt64 binlog_position_, String binlog_do_db_, String binlog_ignore_db_, String executed_gtid_set_); + std::vector need_dumping_tables; + + MasterStatusInfo(mysqlxx::PoolWithFailover::Entry & connection, const String & database); + + void fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & connection); }; diff --git a/src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp b/src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp index c52588cab64..c40cbaf846a 100644 --- a/src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp +++ b/src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp @@ -83,7 +83,7 @@ void CreateQueryMatcher::visit(ASTPtr & ast, Data & data) visit(*t, ast, data); } -void CreateQueryMatcher::visit(MySQLParser::ASTCreateQuery & create, ASTPtr &, Data & data) +void CreateQueryMatcher::visit(const MySQLParser::ASTCreateQuery & create, const ASTPtr &, Data & data) { if (create.like_table) throw Exception("Cannot convert create like statement to ClickHouse SQL", ErrorCodes::NOT_IMPLEMENTED); @@ -104,7 +104,13 @@ void CreateQueryMatcher::visit(MySQLParser::ASTCreateQuery & create, ASTPtr &, D << " ORDER BY " << queryToString(data.getFormattedOrderByExpression()); } -void CreateQueryMatcher::visit(MySQLParser::ASTCreateDefines & create_defines, ASTPtr &, Data & data) +void CreateQueryMatcher::visit(const MySQLParser::ASTDeclareIndex & declare_index, const ASTPtr &, Data & data) +{ + if (startsWith(declare_index.index_type, "PRIMARY_KEY_")) + data.addPrimaryKey(declare_index.index_columns); +} + +void CreateQueryMatcher::visit(const MySQLParser::ASTCreateDefines & create_defines, const ASTPtr &, Data & data) { if (!create_defines.columns || create_defines.columns->children.empty()) throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); @@ -119,13 +125,7 @@ void CreateQueryMatcher::visit(MySQLParser::ASTCreateDefines & create_defines, A visit(*column->as(), column, data); } -void CreateQueryMatcher::visit(const MySQLParser::ASTDeclareIndex & declare_index, ASTPtr &, Data & data) -{ - if (startsWith(declare_index.index_type, "PRIMARY_KEY_")) - data.addPrimaryKey(declare_index.index_columns); -} - -void CreateQueryMatcher::visit(const MySQLParser::ASTDeclareColumn & declare_column, ASTPtr &, Data & data) +void CreateQueryMatcher::visit(const MySQLParser::ASTDeclareColumn & declare_column, const ASTPtr &, Data & data) { if (!declare_column.data_type) throw Exception("Missing type in definition of column.", ErrorCodes::UNKNOWN_TYPE); @@ -158,7 +158,7 @@ void CreateQueryMatcher::visit(const MySQLParser::ASTDeclareColumn & declare_col throw Exception("Unsupported MySQL data type " + queryToString(declare_column.data_type) + ".", ErrorCodes::NOT_IMPLEMENTED); } -void CreateQueryMatcher::visit(const MySQLParser::ASTDeclarePartitionOptions & declare_partition_options, ASTPtr &, Data & data) +void CreateQueryMatcher::visit(const MySQLParser::ASTDeclarePartitionOptions & declare_partition_options, const ASTPtr &, Data & data) { data.addPartitionKey(declare_partition_options.partition_expression); } diff --git a/src/Interpreters/MySQL/CreateQueryConvertVisitor.h b/src/Interpreters/MySQL/CreateQueryConvertVisitor.h index c4c31f54b88..10fcb0e7b7e 100644 --- a/src/Interpreters/MySQL/CreateQueryConvertVisitor.h +++ b/src/Interpreters/MySQL/CreateQueryConvertVisitor.h @@ -45,29 +45,21 @@ public: static bool needChildVisit(ASTPtr &, const ASTPtr &) { return false; } private: - static void visit(MySQLParser::ASTCreateQuery & create, ASTPtr & ast, Data &); + static void visit(const MySQLParser::ASTCreateQuery & create, const ASTPtr &, Data & data); - static void visit(MySQLParser::ASTCreateDefines & create_defines, ASTPtr & ast, Data & data); + static void visit(const MySQLParser::ASTDeclareIndex & declare_index, const ASTPtr &, Data & data); - static void visit(const MySQLParser::ASTDeclareIndex & declare_index, ASTPtr & ast, Data & data); + static void visit(const MySQLParser::ASTCreateDefines & create_defines, const ASTPtr &, Data & data); - static void visit(const MySQLParser::ASTDeclareColumn & declare_column, ASTPtr & ast, Data & data); - - static void visit(const MySQLParser::ASTDeclarePartitionOptions & declare_partition_options, ASTPtr & ast, Data & data); - -// static void visitPartitionBy(MySQLParser::ASTCreateQuery & create, ASTPtr & ast, Data & data); - -// static void visitPartitionBy(MySQLParser::ASTCreateDefines & create_defines, ASTPtr & ast, Data & data); - -// static void visitPartitionBy(const MySQLParser::ASTDeclarePartitionOptions & partition_options, ASTPtr & ast, Data & data); - -// static void visitColumns(const ASTFunction & declare_column, ASTPtr & ast, Data & data); -// static void visit(ASTTableJoin & join, const ASTPtr & ast, Data &); + static void visit(const MySQLParser::ASTDeclareColumn & declare_column, const ASTPtr &, Data & data); + static void visit(const MySQLParser::ASTDeclarePartitionOptions & declare_partition_options, const ASTPtr &, Data & data); }; using CreateQueryConvertVisitor = CreateQueryMatcher::Visitor; } + + } From 0c52d425ba4d96d8b01ddc881a4e62c408cfc5f1 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 29 May 2020 10:39:22 +0800 Subject: [PATCH 066/374] ISSUES-4006 suport synchronous MySQL incremental data --- src/Core/MySQLClient.cpp | 6 + src/Core/MySQLClient.h | 2 + .../AddingVersionsBlockOutputStream.cpp | 55 +++++ .../AddingVersionsBlockOutputStream.h | 30 +++ src/Databases/DatabaseFactory.cpp | 7 +- src/Databases/MySQL/DataBuffers.cpp | 135 +++++++++++ src/Databases/MySQL/DataBuffers.h | 42 ++++ .../MySQL/DatabaseMaterializeMySQL.cpp | 197 ++++++++++------ .../MySQL/DatabaseMaterializeMySQL.h | 151 ++++++++++-- src/Databases/MySQL/MasterStatusInfo.cpp | 146 ++++++++---- src/Databases/MySQL/MasterStatusInfo.h | 11 +- src/Databases/MySQL/queryConvert.cpp | 152 +++++++++++++ src/Databases/MySQL/queryConvert.h | 19 ++ ...vertVisitor.cpp => CreateQueryVisitor.cpp} | 137 +++-------- ...yConvertVisitor.h => CreateQueryVisitor.h} | 39 ++-- .../gtest_create_query_convert_visitor.cpp | 215 ++++++++++-------- 16 files changed, 1011 insertions(+), 333 deletions(-) create mode 100644 src/DataStreams/AddingVersionsBlockOutputStream.cpp create mode 100644 src/DataStreams/AddingVersionsBlockOutputStream.h create mode 100644 src/Databases/MySQL/DataBuffers.cpp create mode 100644 src/Databases/MySQL/DataBuffers.h create mode 100644 src/Databases/MySQL/queryConvert.cpp create mode 100644 src/Databases/MySQL/queryConvert.h rename src/Interpreters/MySQL/{CreateQueryConvertVisitor.cpp => CreateQueryVisitor.cpp} (55%) rename src/Interpreters/MySQL/{CreateQueryConvertVisitor.h => CreateQueryVisitor.h} (60%) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 4f3186fd2f7..5905de11dd2 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -17,6 +17,12 @@ MySQLClient::MySQLClient(const String & host_, UInt16 port_, const String & user client_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_PLUGIN_AUTH | CLIENT_SECURE_CONNECTION; } +MySQLClient::MySQLClient(MySQLClient && other) + : host(other.host), port(other.port), user(other.user), password(other.password) + , client_capability_flags(other.client_capability_flags) +{ +} + void MySQLClient::connect() { if (connected) diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 726707dd129..5c42e5c5d34 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -28,6 +28,8 @@ class MySQLClient { public: MySQLClient(const String & host_, UInt16 port_, const String & user_, const String & password_); + MySQLClient(MySQLClient && other); + void connect(); void disconnect(); void ping(); diff --git a/src/DataStreams/AddingVersionsBlockOutputStream.cpp b/src/DataStreams/AddingVersionsBlockOutputStream.cpp new file mode 100644 index 00000000000..2adc1488e98 --- /dev/null +++ b/src/DataStreams/AddingVersionsBlockOutputStream.cpp @@ -0,0 +1,55 @@ +#include + +#include +#include + +namespace DB +{ + +void AddingVersionsBlockOutputStream::writePrefix() +{ + output->writePrefix(); +} + +void AddingVersionsBlockOutputStream::writeSuffix() +{ + output->writeSuffix(); +} + +void AddingVersionsBlockOutputStream::flush() +{ + output->flush(); +} + +void AddingVersionsBlockOutputStream::write(const Block & block) +{ + /// create_version and delete_version are always in the last place + Block res; + size_t rows = block.rows(); + + for (size_t index = 0; index < block.columns(); ++index) + res.insert(block.getByPosition(index)); + + DataTypePtr data_type = std::make_shared(); + + auto create_version = ColumnUInt64::create(rows); + for (size_t index = 0; index < rows; ++index) + create_version->getData()[index] = ++version; + + Block header = output->getHeader(); + res.insert(ColumnWithTypeAndName(create_version->getPtr(), data_type, header.getByPosition(header.columns() - 2).name)); + res.insert(ColumnWithTypeAndName(data_type->createColumnConstWithDefaultValue(rows)->convertToFullColumnIfConst(), data_type, header.getByPosition(header.columns() - 1).name)); + output->write(res); +} +Block AddingVersionsBlockOutputStream::getHeader() const +{ + Block res; + Block header = output->getHeader(); + + for (size_t index = 0; index < header.columns() - 2; ++index) + res.insert(header.getByPosition(index)); + + return res; +} + +} diff --git a/src/DataStreams/AddingVersionsBlockOutputStream.h b/src/DataStreams/AddingVersionsBlockOutputStream.h new file mode 100644 index 00000000000..cd6eb84e9bc --- /dev/null +++ b/src/DataStreams/AddingVersionsBlockOutputStream.h @@ -0,0 +1,30 @@ +#pragma once + +#include + +namespace DB +{ + +class AddingVersionsBlockOutputStream : public IBlockOutputStream +{ +public: + AddingVersionsBlockOutputStream(size_t & version_, const BlockOutputStreamPtr & output_) + : version(version_), output(output_) + { + } + + Block getHeader() const override; + + void write(const Block & block) override; + + void flush() override; + + void writePrefix() override; + void writeSuffix() override; + +private: + size_t & version; + BlockOutputStreamPtr output; +}; + +} diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index abfbfbd8878..87a8e6d775a 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -23,6 +23,7 @@ # include # include # include +# include #endif namespace DB @@ -118,8 +119,12 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String auto mysql_pool = mysqlxx::Pool(mysql_database_name, remote_host_name, mysql_user_name, mysql_user_password, remote_port); if (materializeMySQLDatabase(engine_define->settings)) + { + MySQLClient client(remote_host_name, remote_port, mysql_user_name, mysql_user_password); + return std::make_shared( - context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_pool)); + context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_pool), std::move(client)); + } return std::make_shared(context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_pool)); } diff --git a/src/Databases/MySQL/DataBuffers.cpp b/src/Databases/MySQL/DataBuffers.cpp new file mode 100644 index 00000000000..8659eadb8bb --- /dev/null +++ b/src/Databases/MySQL/DataBuffers.cpp @@ -0,0 +1,135 @@ +#include + +#include +#include + +namespace DB +{ + +DataBuffers::DataBuffers(size_t & version_, IDatabase * database_, const std::function &)> & flush_function_) + : version(version_), database(database_), flush_function(flush_function_) +{ + + /// TODO: 定时刷新 +} + +void DataBuffers::flush() +{ + flush_function(buffers); + buffers.clear(); +} + +void DataBuffers::writeData(const std::string & table_name, const std::vector & rows_data) +{ + Block & to = getTableBuffer(table_name, rows_data.size()); + for (size_t column = 0; column < to.columns() - 2; ++column) + { + /// normally columns + MutableColumnPtr col_to = (*std::move(to.getByPosition(column).column)).mutate(); + + for (size_t index = 0; index < rows_data.size(); ++index) + col_to->insert(DB::get(rows_data[index])[column]); + } + + Field new_version(UInt64(++version)); + MutableColumnPtr create_version_column = (*std::move(to.getByPosition(to.columns() - 2)).column).mutate(); + MutableColumnPtr delete_version_column = (*std::move(to.getByPosition(to.columns() - 1)).column).mutate(); + + delete_version_column->insertManyDefaults(rows_data.size()); + + for (size_t index = 0; index < rows_data.size(); ++index) + create_version_column->insert(new_version); +} + +void DataBuffers::updateData(const String & table_name, const std::vector & rows_data) +{ + if (rows_data.size() % 2 != 0) + throw Exception("LOGICAL ERROR: ", ErrorCodes::LOGICAL_ERROR); + + Block & to = getTableBuffer(table_name, rows_data.size()); + for (size_t column = 0; column < to.columns() - 2; ++column) + { + /// normally columns + MutableColumnPtr col_to = (*std::move(to.getByPosition(column).column)).mutate(); + + for (size_t index = 0; index < rows_data.size(); ++index) + col_to->insert(DB::get(rows_data[index])[column]); + } + + Field new_version(UInt64(++version)); + MutableColumnPtr create_version_column = (*std::move(to.getByPosition(to.columns() - 2)).column).mutate(); + MutableColumnPtr delete_version_column = (*std::move(to.getByPosition(to.columns() - 1)).column).mutate(); + + for (size_t index = 0; index < rows_data.size(); ++index) + { + if (index % 2 == 0) + { + create_version_column->insertDefault(); + delete_version_column->insert(new_version); + } + else + { + delete_version_column->insertDefault(); + create_version_column->insert(new_version); + } + } +} + +void DataBuffers::deleteData(const String & table_name, const std::vector & rows_data) +{ + Block & to = getTableBuffer(table_name, rows_data.size()); + for (size_t column = 0; column < to.columns() - 2; ++column) + { + /// normally columns + MutableColumnPtr col_to = (*std::move(to.getByPosition(column).column)).mutate(); + + for (size_t index = 0; index < rows_data.size(); ++index) + col_to->insert(DB::get(rows_data[index])[column]); + } + + Field new_version(UInt64(++version)); + MutableColumnPtr create_version_column = (*std::move(to.getByPosition(to.columns() - 2)).column).mutate(); + MutableColumnPtr delete_version_column = (*std::move(to.getByPosition(to.columns() - 1)).column).mutate(); + + create_version_column->insertManyDefaults(rows_data.size()); + + for (size_t index = 0; index < rows_data.size(); ++index) + delete_version_column->insert(new_version); +} + +Block & DataBuffers::getTableBuffer(const String & table_name, size_t write_size) +{ + if (buffers.find(table_name) == buffers.end()) + { + StoragePtr write_storage = database->tryGetTable(table_name); + buffers[table_name] = write_storage->getSampleBlockNonMaterialized(); + } + + /// TODO: settings + if (buffers[table_name].rows() + write_size > 8192) + flush(); + + return buffers[table_name]; +} + +[[noreturn]] void DataBuffers::scheduleFlush() +{ + while (1) + { + try + { + flush(); + sleepForSeconds(1); + } + catch (...) + { +// ++error_count; +// sleep_time = std::min( +// std::chrono::milliseconds{Int64(default_sleep_time.count() * std::exp2(error_count))}, +// max_sleep_time); + tryLogCurrentException(""); + } + } +} + +} diff --git a/src/Databases/MySQL/DataBuffers.h b/src/Databases/MySQL/DataBuffers.h new file mode 100644 index 00000000000..4e69eb438d7 --- /dev/null +++ b/src/Databases/MySQL/DataBuffers.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class DataBuffers : private boost::noncopyable +{ +public: + DataBuffers(size_t & version_, IDatabase * database_, const std::function &)> & flush_function_); + + void flush(); + + void writeData(const std::string & table_name, const std::vector & rows_data); + + void updateData(const std::string & table_name, const std::vector & rows_data); + + void deleteData(const std::string & table_name, const std::vector & rows_data); + + +private: + size_t & version; + IDatabase * database; + std::function &)> flush_function; + + mutable std::mutex mutex; + std::unordered_map buffers; + + [[noreturn]] void scheduleFlush(); + + Block & getTableBuffer(const String & table_name, size_t write_size); + + ThreadFromGlobalPool thread{&DataBuffers::scheduleFlush, this}; +}; + +} diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 508cdd5a861..63676e74f30 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -6,21 +6,25 @@ #include +# include +# include +# include # include # include -# include +# include +# include # include +# include # include -# include # include # include -# include +# include # include # include # include -# include # include # include +# include namespace DB { @@ -32,26 +36,25 @@ namespace ErrorCodes DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( const Context & context, const String & database_name_, const String & metadata_path_, - const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_) - : DatabaseOrdinary(database_name_, metadata_path_, context) - /*, global_context(context.getGlobalContext()), metadata_path(metadata_path_)*/ - , database_engine_define(database_engine_define_->clone()), mysql_database_name(mysql_database_name_), pool(std::move(pool_)) + const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_) + : IDatabase(database_name_), global_context(context.getGlobalContext()), metadata_path(metadata_path_) + , database_engine_define(database_engine_define_->clone()), mysql_database_name(mysql_database_name_) + , nested_database(std::make_shared(database_name_, metadata_path, global_context)) + , pool(std::move(pool_)), client(std::move(client_)), log(&Logger::get("DatabaseMaterializeMySQL")) { /// TODO: 做简单的check, 失败即报错 } -void DatabaseMaterializeMySQL::tryToExecuteQuery(const String & query_to_execute) +BlockIO DatabaseMaterializeMySQL::tryToExecuteQuery(const String & query_to_execute, const String & comment) { - ReadBufferFromString istr(query_to_execute); - String dummy_string; - WriteBufferFromString ostr(dummy_string); + String query_prefix = "/*" + comment + " for " + backQuoteIfNeed(database_name) + " Database */ "; try { Context context = global_context; context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; context.setCurrentQueryId(""); // generate random query_id - executeQuery(istr, ostr, false, context, {}); + return executeQuery(query_prefix + query_to_execute, context, true); } catch (...) { @@ -62,7 +65,7 @@ void DatabaseMaterializeMySQL::tryToExecuteQuery(const String & query_to_execute LOG_DEBUG(log, "Executed query: " << query_to_execute); } -String DatabaseMaterializeMySQL::getCreateQuery(const mysqlxx::Pool::Entry & connection, const String & database, const String & table_name) +String DatabaseMaterializeMySQL::getCreateQuery(const mysqlxx::Pool::Entry & connection, const String & table_name) { Block show_create_table_header{ {std::make_shared(), "Table"}, @@ -77,57 +80,82 @@ String DatabaseMaterializeMySQL::getCreateQuery(const mysqlxx::Pool::Entry & con if (!create_query_block || create_query_block.rows() != 1) throw Exception("LOGICAL ERROR mysql show create return more rows.", ErrorCodes::LOGICAL_ERROR); - const auto & create_query = create_query_block.getByName("Create Table").column->getDataAt(0); - - MySQLParser::ParserCreateQuery p_create_query; - ASTPtr ast = parseQuery(p_create_query, create_query.data, create_query.data + create_query.size, "", 0, 0); - - if (!ast || !ast->as()) - throw Exception("LOGICAL ERROR: ast cannot cast to MySQLParser::ASTCreateQuery.", ErrorCodes::LOGICAL_ERROR); - - WriteBufferFromOwnString out; - ast->as()->database = database; - MySQLVisitor::CreateQueryConvertVisitor::Data data{.out = out, .context = global_context}; - MySQLVisitor::CreateQueryConvertVisitor visitor(data); - visitor.visit(ast); - return out.str(); + return create_query_block.getByName("Create Table").column->getDataAt(0).toString(); } -void DatabaseMaterializeMySQL::dumpMySQLDatabase(const std::function & is_cancelled) +BlockOutputStreamPtr DatabaseMaterializeMySQL::getTableOutput(const String & table_name, bool fill_version) { - mysqlxx::PoolWithFailover::Entry connection = pool.get(); + Context context = global_context; + context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + context.setCurrentQueryId(""); // generate random query_id - MasterStatusInfo info(connection, mysql_database_name); + StoragePtr write_storage = nested_database->tryGetTable(table_name); + auto table_lock = write_storage->lockStructureForShare(true, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); - for (const auto & dumping_table_name : info.need_dumping_tables) + BlockOutputStreamPtr output = write_storage->write(ASTPtr{}, global_context); + output->addTableLock(table_lock); + + return fill_version ? std::make_shared(version, output) : output; +} + +void DatabaseMaterializeMySQL::dumpDataForTables(mysqlxx::Pool::Entry & connection, const std::vector & tables_name, const std::function & is_cancelled) +{ + std::unordered_map tables_create_query; + for (size_t index = 0; index < tables_name.size() && !is_cancelled(); ++index) + tables_create_query[tables_name[index]] = getCreateQuery(connection, tables_name[index]); + + auto iterator = tables_create_query.begin(); + for (; iterator != tables_create_query.end() && !is_cancelled(); ++iterator) { - if (is_cancelled()) - return; + const auto & table_name = iterator->first; + MySQLTableStruct table_struct = visitCreateQuery(iterator->second, global_context, database_name); + String comment = String("Dumping ") + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name); + tryToExecuteQuery(toCreateQuery(table_struct, global_context), comment); - const auto & table_name = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(dumping_table_name); - String query_prefix = "/* Dumping " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(dumping_table_name) + " for " - + backQuoteIfNeed(database_name) + " Database */ "; - - tryToExecuteQuery(query_prefix + " DROP TABLE IF EXISTS " + table_name); - tryToExecuteQuery(query_prefix + getCreateQuery(connection, database_name, dumping_table_name)); - - Context context = global_context; - context.setCurrentQueryId(""); // generate random query_id - context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - BlockIO streams = executeQuery(query_prefix + " INSERT INTO " + table_name + " VALUES", context, true); - - if (!streams.out) - throw Exception("LOGICAL ERROR out stream is undefined.", ErrorCodes::LOGICAL_ERROR); - - MySQLBlockInputStream input( - connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(dumping_table_name), - streams.out->getHeader(), DEFAULT_BLOCK_SIZE); - - copyData(input, *streams.out, is_cancelled); - /// TODO: 启动slave, 监听事件 + BlockOutputStreamPtr out = getTableOutput(table_name, true); + MySQLBlockInputStream input(connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name), out->getHeader(), DEFAULT_BLOCK_SIZE); + copyData(input, *out, is_cancelled); } } -void DatabaseMaterializeMySQL::synchronization() + +MasterStatusInfo DatabaseMaterializeMySQL::prepareSynchronized(std::unique_lock & lock) +{ + while(!sync_quit.load(std::memory_order_seq_cst)) + { + try + { + LOG_DEBUG(log, "Checking " + database_name + " database status."); + while (!sync_quit && !DatabaseCatalog::instance().isDatabaseExist(database_name)) + sync_cond.wait_for(lock, std::chrono::seconds(1)); + + LOG_DEBUG(log, database_name + " database status is OK."); + + mysqlxx::PoolWithFailover::Entry connection = pool.get(); + MasterStatusInfo master_info(connection, getMetadataPath() + "/.master_status", mysql_database_name); + + if (!master_info.need_dumping_tables.empty()) + { + /// TODO: 删除所有表结构, 这可能需要考虑到仍然有查询在使用这个表. + dumpDataForTables(connection, master_info.need_dumping_tables, [&]() { return sync_quit.load(std::memory_order_seq_cst); }); + master_info.finishDump(); + } + + client.connect(); + client.startBinlogDump(std::rand(), mysql_database_name, master_info.binlog_file, master_info.binlog_position); + return master_info; + } + catch(...) + { + tryLogCurrentException(log, "Prepare MySQL Synchronized exception and retry"); + + sleepForSeconds(1); + } + } + + throw Exception("", ErrorCodes::LOGICAL_ERROR); +} + +void DatabaseMaterializeMySQL::synchronized() { setThreadName("MySQLDBSync"); @@ -135,18 +163,55 @@ void DatabaseMaterializeMySQL::synchronization() { std::unique_lock lock{sync_mutex}; - LOG_DEBUG(log, "Checking " + database_name + " database status."); - while (!sync_quit && !DatabaseCatalog::instance().isDatabaseExist(database_name)) - sync_cond.wait_for(lock, std::chrono::seconds(1)); + MasterStatusInfo master_status = prepareSynchronized(lock); - LOG_DEBUG(log, database_name + " database status is OK."); - - Poco::File dumped_flag(getMetadataPath() + "/dumped.flag"); - - if (!dumped_flag.exists()) + DataBuffers buffers(version, this, [&](const std::unordered_map & tables_data) { - dumpMySQLDatabase([&]() { return sync_quit.load(std::memory_order_seq_cst); }); - dumped_flag.createFile(); + master_status.transaction(client.getPosition(), [&]() /// At least once, There is only one possible reference: https://github.com/ClickHouse/ClickHouse/pull/8467 + { + for (const auto & [table_name, data] : tables_data) + { + if (!sync_quit.load(std::memory_order_seq_cst)) + { + LOG_DEBUG(log, "Prepare to flush data."); + BlockOutputStreamPtr output = getTableOutput(table_name, false); + output->writePrefix(); + output->write(data); + output->writeSuffix(); + output->flush(); + LOG_DEBUG(log, "Finish data flush."); + } + } + }); + }); + + while (!sync_quit.load(std::memory_order_seq_cst)) + { + const auto & event = client.readOneBinlogEvent(); + + if (event->type() == MYSQL_WRITE_ROWS_EVENT) + { + WriteRowsEvent & write_rows_event = static_cast(*event); + write_rows_event.dump(); + buffers.writeData(write_rows_event.table, write_rows_event.rows); + } + else if (event->type() == MYSQL_UPDATE_ROWS_EVENT) + { + UpdateRowsEvent & update_rows_event = static_cast(*event); + update_rows_event.dump(); + buffers.updateData(update_rows_event.table, update_rows_event.rows); + } + else if (event->type() == MYSQL_DELETE_ROWS_EVENT) + { + DeleteRowsEvent & delete_rows_event = static_cast(*event); + delete_rows_event.dump(); + buffers.deleteData(delete_rows_event.table, delete_rows_event.rows); + } + else if (event->type() == MYSQL_QUERY_EVENT) + { + /// TODO: 识别, 查看是否支持的DDL, 支持的话立即刷新当前的数据, 然后执行DDL. + buffers.flush(); + } } } catch(...) diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h index b4fabdb20eb..b6b27d33cae 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -3,46 +3,165 @@ #include "config_core.h" #if USE_MYSQL -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include namespace DB { -class DatabaseMaterializeMySQL : public DatabaseOrdinary +class DatabaseMaterializeMySQL : public IDatabase { public: DatabaseMaterializeMySQL( const Context & context, const String & database_name_, const String & metadata_path_, - const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_); + const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_, + MySQLClient && client_); String getEngineName() const override { return "MySQL"; } + void shutdown() override { nested_database->shutdown(); } + + bool empty() const override { return nested_database->empty(); } + + String getDataPath() const override { return nested_database->getDataPath(); } + + String getTableDataPath(const String & string) const override { return nested_database->getTableDataPath(string); } + + String getTableDataPath(const ASTCreateQuery & query) const override { return nested_database->getTableDataPath(query); } + + UUID tryGetTableUUID(const String & string) const override { return nested_database->tryGetTableUUID(string); } + + bool isDictionaryExist(const String & string) const override { return nested_database->isDictionaryExist(string); } + + DatabaseDictionariesIteratorPtr getDictionariesIterator(const FilterByNameFunction & filter_by_dictionary_name) override + { + return nested_database->getDictionariesIterator(filter_by_dictionary_name); + } + + void createTable(const Context & context, const String & string, const StoragePtr & ptr, const ASTPtr & astPtr) override + { + nested_database->createTable(context, string, ptr, astPtr); + } + + void createDictionary(const Context & context, const String & string, const ASTPtr & ptr) override + { + nested_database->createDictionary(context, string, ptr); + } + + void dropTable(const Context & context, const String & string, bool no_delay) override + { + nested_database->dropTable(context, string, no_delay); + } + + void removeDictionary(const Context & context, const String & string) override { nested_database->removeDictionary(context, string); } + + void attachTable(const String & string, const StoragePtr & ptr, const String & relative_table_path) override + { + nested_database->attachTable(string, ptr, relative_table_path); + } + + void attachDictionary(const String & string, const DictionaryAttachInfo & info) override { nested_database->attachDictionary(string, info); } + + StoragePtr detachTable(const String & string) override { return nested_database->detachTable(string); } + + void detachDictionary(const String & string) override { nested_database->detachDictionary(string); } + + void renameTable(const Context & context, const String & string, IDatabase & database, const String & string1, bool b) override + { + nested_database->renameTable(context, string, database, string1, b); + } + + void alterTable(const Context & context, const StorageID & id, const StorageInMemoryMetadata & metadata) override + { + nested_database->alterTable(context, id, metadata); + } + + time_t getObjectMetadataModificationTime(const String & string) const override + { + return nested_database->getObjectMetadataModificationTime(string); + } + + Poco::AutoPtr getDictionaryConfiguration(const String & string) const override + { + return nested_database->getDictionaryConfiguration(string); + } + + String getMetadataPath() const override { return nested_database->getMetadataPath(); } + + String getObjectMetadataPath(const String & string) const override { return nested_database->getObjectMetadataPath(string); } + + bool shouldBeEmptyOnDetach() const override { return nested_database->shouldBeEmptyOnDetach(); } + + void drop(const Context & context) override { nested_database->drop(context); } + + bool isTableExist(const String & name) const override { return nested_database->isTableExist(name); } + + StoragePtr tryGetTable(const String & name) const override { return nested_database->tryGetTable(name); } + + void loadStoredObjects(Context & context, bool b) override + { + try + { + LOG_DEBUG(log, "Loading MySQL nested database stored objects."); + nested_database->loadStoredObjects(context, b); + LOG_DEBUG(log, "Loaded MySQL nested database stored objects."); + } + catch (...) + { + tryLogCurrentException(log, "Cannot load MySQL nested database stored objects."); + throw; + } + } + + ASTPtr getCreateDatabaseQuery() const override + { + const auto & create_query = std::make_shared(); + create_query->database = database_name; + create_query->set(create_query->storage, database_engine_define); + return create_query; + } + + DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override { return nested_database->getTablesIterator(filter_by_table_name); } + private: -// const Context & global_context; -// String metadata_path; + const Context & global_context; + String metadata_path; ASTPtr database_engine_define; String mysql_database_name; + DatabasePtr nested_database; + size_t version{0}; mutable mysqlxx::Pool pool; + mutable MySQLClient client; - void synchronization(); + Poco::Logger * log; - void tryToExecuteQuery(const String & query_to_execute); + void synchronized(); - void dumpMySQLDatabase(const std::function & is_cancelled); + MasterStatusInfo prepareSynchronized(std::unique_lock & lock); - String getCreateQuery(const mysqlxx::Pool::Entry & connection, const String & database, const String & table_name); + BlockOutputStreamPtr getTableOutput(const String & table_name, bool fill_version); + + BlockIO tryToExecuteQuery(const String & query_to_execute, const String & comment); + + String getCreateQuery(const mysqlxx::Pool::Entry & connection, const String & table_name); + + void dumpDataForTables(mysqlxx::Pool::Entry & connection, const std::vector & tables_name, const std::function & is_cancelled); mutable std::mutex sync_mutex; std::atomic sync_quit{false}; std::condition_variable sync_cond; - ThreadFromGlobalPool thread{&DatabaseMaterializeMySQL::synchronization, this}; + ThreadFromGlobalPool thread{&DatabaseMaterializeMySQL::synchronized, this}; }; } diff --git a/src/Databases/MySQL/MasterStatusInfo.cpp b/src/Databases/MySQL/MasterStatusInfo.cpp index 3a1a6c3d7dd..687a028ebdc 100644 --- a/src/Databases/MySQL/MasterStatusInfo.cpp +++ b/src/Databases/MySQL/MasterStatusInfo.cpp @@ -4,15 +4,13 @@ #include #include #include +#include +#include +#include +#include namespace DB { -/*MasterStatusInfo::MasterStatusInfo( - String binlog_file_, UInt64 binlog_position_, String binlog_do_db_, String binlog_ignore_db_, String executed_gtid_set_) - : binlog_file(binlog_file_), binlog_position(binlog_position_), binlog_do_db(binlog_do_db_), binlog_ignore_db(binlog_ignore_db_), - executed_gtid_set(executed_gtid_set_) -{ -}*/ static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entry & connection, const std::string & database) { @@ -31,42 +29,15 @@ static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entr return tables_in_db; } - -MasterStatusInfo::MasterStatusInfo(mysqlxx::PoolWithFailover::Entry & connection, const String & database) -{ - bool locked_tables = false; - - try - { - connection->query("FLUSH TABLES;").execute(); - connection->query("FLUSH TABLES WITH READ LOCK;").execute(); - - locked_tables = true; - fetchMasterStatus(connection); - connection->query("SET SESSION TRANSACTION ISOLATION LEVEL REPEATABLE READ;").execute(); - connection->query("START TRANSACTION /*!40100 WITH CONSISTENT SNAPSHOT */;").execute(); - - need_dumping_tables = fetchTablesInDB(connection, database); - connection->query("UNLOCK TABLES;").execute(); - } - catch (...) - { - if (locked_tables) - connection->query("UNLOCK TABLES;").execute(); - - throw; - } -} void MasterStatusInfo::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & connection) { - Block header - { - {std::make_shared(), "File"}, - {std::make_shared(), "Position"}, - {std::make_shared(), "Binlog_Do_DB"}, - {std::make_shared(), "Binlog_Ignore_DB"}, - {std::make_shared(), "Executed_Gtid_Set"}, - }; + Block header{ + {std::make_shared(), "File"}, + {std::make_shared(), "Position"}, + {std::make_shared(), "Binlog_Do_DB"}, + {std::make_shared(), "Binlog_Ignore_DB"}, + {std::make_shared(), "Executed_Gtid_Set"}, + }; MySQLBlockInputStream input(connection, "SHOW MASTER STATUS;", header, DEFAULT_BLOCK_SIZE); Block master_status = input.read(); @@ -81,4 +52,99 @@ void MasterStatusInfo::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & conn executed_gtid_set = (*master_status.getByPosition(4).column)[0].safeGet(); } +bool MasterStatusInfo::checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection) +{ + Block header{ + {std::make_shared(), "Log_name"}, + {std::make_shared(), "File_size"}, + {std::make_shared(), "Encrypted"} + }; + + MySQLBlockInputStream input(connection, "SHOW MASTER LOGS", header, DEFAULT_BLOCK_SIZE); + + while (Block block = input.read()) + { + for (size_t index = 0; index < block.rows(); ++index) + { + const auto & log_name = (*block.getByPosition(0).column)[index].safeGet(); + if (log_name == binlog_file) + return true; + } + } + return false; +} +void MasterStatusInfo::finishDump() +{ + WriteBufferFromFile out(persistent_path); + out << "Version:\t1\n" + << "Binlog File:\t" << binlog_file << "\nBinlog Position:\t" << binlog_position << "\nBinlog Do DB:\t" << binlog_do_db + << "\nBinlog Ignore DB:\t" << binlog_ignore_db << "\nExecuted GTID SET:\t" << executed_gtid_set; + + out.next(); + out.sync(); +} + +void MasterStatusInfo::transaction(const MySQLReplication::Position & position, const std::function & fun) +{ + binlog_file = position.binlog_name; + binlog_position = position.binlog_pos; + + { + Poco::File temp_file(persistent_path + ".temp"); + if (temp_file.exists()) + temp_file.remove(); + } + + WriteBufferFromFile out(persistent_path + ".temp"); + out << "Version:\t1\n" + << "Binlog File:\t" << binlog_file << "\nBinlog Position:\t" << binlog_position << "\nBinlog Do DB:\t" << binlog_do_db + << "\nBinlog Ignore DB:\t" << binlog_ignore_db << "\nExecuted GTID SET:\t" << executed_gtid_set; + out.next(); + out.sync(); + + fun(); + Poco::File(persistent_path + ".temp").renameTo(persistent_path); +} + +MasterStatusInfo::MasterStatusInfo(mysqlxx::PoolWithFailover::Entry & connection, const String & path_, const String & database) + : persistent_path(path_) +{ + if (Poco::File(persistent_path).exists()) + { + ReadBufferFromFile in(persistent_path); + in >> "Version:\t1\n" >> "Binlog File:\t" >> binlog_file >> "\nBinlog Position:\t" >> binlog_position >> "\nBinlog Do DB:\t" + >> binlog_do_db >> "\nBinlog Ignore DB:\t" >> binlog_ignore_db >> "\nExecuted GTID SET:\t" >> executed_gtid_set; + + if (checkBinlogFileExists(connection)) + { + std::cout << "Load From File \n"; + return; + } + } + + bool locked_tables = false; + + try + { + connection->query("FLUSH TABLES;").execute(); + connection->query("FLUSH TABLES WITH READ LOCK;").execute(); + + locked_tables = true; + fetchMasterStatus(connection); + connection->query("SET SESSION TRANSACTION ISOLATION LEVEL REPEATABLE READ;").execute(); + connection->query("START TRANSACTION /*!40100 WITH CONSISTENT SNAPSHOT */;").execute(); + + need_dumping_tables = fetchTablesInDB(connection, database); + connection->query("UNLOCK TABLES;").execute(); + /// TODO: 拉取建表语句, 解析并构建出表结构(列列表, 主键, 唯一索引, 分区键) + } + catch (...) + { + if (locked_tables) + connection->query("UNLOCK TABLES;").execute(); + + throw; + } +} + } diff --git a/src/Databases/MySQL/MasterStatusInfo.h b/src/Databases/MySQL/MasterStatusInfo.h index ebc2a8c8d12..d18caf4b5fe 100644 --- a/src/Databases/MySQL/MasterStatusInfo.h +++ b/src/Databases/MySQL/MasterStatusInfo.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -9,6 +10,8 @@ namespace DB struct MasterStatusInfo { + const String persistent_path; + String binlog_file; UInt64 binlog_position; String binlog_do_db; @@ -17,10 +20,16 @@ struct MasterStatusInfo std::vector need_dumping_tables; - MasterStatusInfo(mysqlxx::PoolWithFailover::Entry & connection, const String & database); + void finishDump(); void fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & connection); + bool checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection); + + void transaction(const MySQLReplication::Position & position, const std::function & fun); + + MasterStatusInfo(mysqlxx::PoolWithFailover::Entry & connection, const String & path, const String & database); + }; diff --git a/src/Databases/MySQL/queryConvert.cpp b/src/Databases/MySQL/queryConvert.cpp new file mode 100644 index 00000000000..decbf8f3cb6 --- /dev/null +++ b/src/Databases/MySQL/queryConvert.cpp @@ -0,0 +1,152 @@ +#include + +#include +#include + +namespace DB +{ + +ASTPtr getFormattedOrderByExpression(const MySQLTableStruct & table_struct) +{ + if (table_struct.primary_keys.empty()) + return makeASTFunction("tuple"); + + /// TODO: support unique key & key + const auto function = std::make_shared(); + function->name = "tuple"; + function->arguments = std::make_shared(); + function->children.push_back(function->arguments); + function->arguments->children = table_struct.primary_keys; + return function; +} + +template +Field choiceBetterRangeSize(TType min, TType max, size_t max_ranges, size_t min_size_pre_range) +{ + UInt64 interval = UInt64(max) - min; + size_t calc_rows_pre_range = std::ceil(interval / double(max_ranges)); + size_t rows_pre_range = std::max(min_size_pre_range, calc_rows_pre_range); + + if (rows_pre_range >= interval) + return Null(); + + return rows_pre_range > std::numeric_limits::max() ? Field(UInt64(rows_pre_range)) : Field(TType(rows_pre_range)); +} + +ASTPtr getFormattedPartitionByExpression(const MySQLTableStruct & table_struct, const Context & context, size_t max_ranges, size_t min_rows_pre_range) +{ + ASTPtr partition_columns = std::make_shared(); + + if (!table_struct.partition_keys.empty()) + partition_columns->children = table_struct.partition_keys; + else if (!table_struct.primary_keys.empty()) + { + ASTPtr expr_list = std::make_shared(); + expr_list->children = table_struct.primary_keys; + + auto syntax = SyntaxAnalyzer(context).analyze(expr_list, table_struct.columns_name_and_type); + auto index_expr = ExpressionAnalyzer(expr_list, syntax, context).getActions(false); + const NamesAndTypesList & required_names_and_types = index_expr->getRequiredColumnsWithTypes(); + + const auto & addPartitionColumn = [&](const String & column_name, const DataTypePtr & type, Field better_pre_range_size) + { + partition_columns->children.emplace_back(std::make_shared(column_name)); + + if (type->isNullable()) + partition_columns->children.back() = makeASTFunction("assumeNotNull", partition_columns->children.back()); + + if (!better_pre_range_size.isNull()) + partition_columns->children.back() + = makeASTFunction("divide", partition_columns->children.back(), std::make_shared(better_pre_range_size)); + }; + + for (const auto & required_name_and_type : required_names_and_types) + { + DataTypePtr assume_not_null = required_name_and_type.type; + if (assume_not_null->isNullable()) + assume_not_null = (static_cast(*assume_not_null)).getNestedType(); + + WhichDataType which(assume_not_null); + if (which.isInt8()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isInt16()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isInt32()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isInt64()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isUInt8()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isUInt16()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isUInt32()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isUInt64()) + addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( + std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); + else if (which.isDateOrDateTime()) + { + partition_columns->children.emplace_back(std::make_shared(required_name_and_type.name)); + + if (required_name_and_type.type->isNullable()) + partition_columns->children.back() = makeASTFunction("assumeNotNull", partition_columns->children.back()); + + partition_columns->children.back() = makeASTFunction("toYYYYMM", partition_columns->children.back()); + } + } + } + + const auto function = std::make_shared(); + function->name = "tuple"; + function->arguments = partition_columns; + function->children.push_back(function->arguments); + return function; +} + +String getUniqueColumnName(NamesAndTypesList columns_name_and_type, const String & prefix) +{ + const auto & is_unique = [&](const String & column_name) + { + for (const auto & column_name_and_type : columns_name_and_type) + { + if (column_name_and_type.name == column_name) + return false; + } + + return true; + }; + + if (is_unique(prefix)) + return prefix; + + for (size_t index = 0; ; ++index) + { + const String & cur_name = prefix + "_" + toString(index); + if (is_unique(cur_name)) + return cur_name; + } +} + +String toCreateQuery(const MySQLTableStruct & table_struct, const Context & context) +{ + /// TODO: settings + String create_version = getUniqueColumnName(table_struct.columns_name_and_type, "_create_version"); + String delete_version = getUniqueColumnName(table_struct.columns_name_and_type, "_delete_version"); + WriteBufferFromOwnString out; + out << "CREATE TABLE " << (table_struct.if_not_exists ? "IF NOT EXISTS" : "") + << backQuoteIfNeed(table_struct.database_name) + "." << backQuoteIfNeed(table_struct.table_name) << "(" + << queryToString(InterpreterCreateQuery::formatColumns(table_struct.columns_name_and_type)) + << ", " << create_version << " UInt64, " << delete_version << " UInt64" << ") ENGINE = MergeTree()" + << " PARTITION BY " << queryToString(getFormattedPartitionByExpression(table_struct, context, 1000, 50000)) + << " ORDER BY " << queryToString(getFormattedOrderByExpression(table_struct)); + return out.str(); +} + +} diff --git a/src/Databases/MySQL/queryConvert.h b/src/Databases/MySQL/queryConvert.h new file mode 100644 index 00000000000..eb5ae5b2648 --- /dev/null +++ b/src/Databases/MySQL/queryConvert.h @@ -0,0 +1,19 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +String toCreateQuery(const MySQLTableStruct & table_struct, const Context & context); + +} diff --git a/src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp b/src/Interpreters/MySQL/CreateQueryVisitor.cpp similarity index 55% rename from src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp rename to src/Interpreters/MySQL/CreateQueryVisitor.cpp index c40cbaf846a..c41b51d54dc 100644 --- a/src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp +++ b/src/Interpreters/MySQL/CreateQueryVisitor.cpp @@ -1,13 +1,13 @@ -#include -#include #include -#include -#include -#include +#include #include +#include +#include #include #include +#include #include +#include #include #include @@ -94,14 +94,9 @@ void CreateQueryMatcher::visit(const MySQLParser::ASTCreateQuery & create, const if (create.partition_options) visit(*create.partition_options->as(), create.partition_options, data); - auto expression_list = std::make_shared(); - expression_list->children = data.primary_keys; - - data.out << "CREATE TABLE " << (create.if_not_exists ? "IF NOT EXISTS" : "") - << (create.database.empty() ? "" : backQuoteIfNeed(create.database) + ".") << backQuoteIfNeed(create.table) - << "(" << queryToString(InterpreterCreateQuery::formatColumns(data.columns_name_and_type)) << ") ENGINE = MergeTree()" - " PARTITION BY " << queryToString(data.getFormattedPartitionByExpression()) - << " ORDER BY " << queryToString(data.getFormattedOrderByExpression()); + data.table_name = create.table; + data.database_name = create.database; + data.if_not_exists = create.if_not_exists; } void CreateQueryMatcher::visit(const MySQLParser::ASTDeclareIndex & declare_index, const ASTPtr &, Data & data) @@ -201,111 +196,39 @@ void CreateQueryMatcher::Data::addPartitionKey(const ASTPtr & partition_key) partition_keys.emplace_back(partition_key); } -ASTPtr CreateQueryMatcher::Data::getFormattedOrderByExpression() +} + +bool MySQLTableStruct::operator==(const MySQLTableStruct & other) const { - if (primary_keys.empty()) - return makeASTFunction("tuple"); + const auto & this_expression = std::make_shared(); + this_expression->children.insert(this_expression->children.begin(), primary_keys.begin(), primary_keys.end()); + this_expression->children.insert(this_expression->children.begin(), partition_keys.begin(), partition_keys.end()); - /// TODO: support unique key & key - const auto function = std::make_shared(); - function->name = "tuple"; - function->arguments = std::make_shared(); - function->children.push_back(function->arguments); - function->arguments->children = primary_keys; + const auto & other_expression = std::make_shared(); + other_expression->children.insert(other_expression->children.begin(), other.primary_keys.begin(), other.primary_keys.end()); + other_expression->children.insert(other_expression->children.begin(), other.partition_keys.begin(), other.partition_keys.end()); - return function; + return queryToString(this_expression) == queryToString(other_expression) && columns_name_and_type == other.columns_name_and_type; } -template -Field choiceBetterRangeSize(TType min, TType max, size_t max_ranges, size_t min_size_pre_range) +MySQLTableStruct visitCreateQuery(ASTPtr & create_query, const Context & context, const std::string & new_database) { - UInt64 interval = UInt64(max) - min; - size_t calc_rows_pre_range = std::ceil(interval / double(max_ranges)); - size_t rows_pre_range = std::max(min_size_pre_range, calc_rows_pre_range); - - if (rows_pre_range >= interval) - return Null(); - - return rows_pre_range > std::numeric_limits::max() ? Field(UInt64(rows_pre_range)) : Field(TType(rows_pre_range)); + create_query->as()->database = new_database; + MySQLVisitor::CreateQueryVisitor::Data table_struct(context); + MySQLVisitor::CreateQueryVisitor visitor(table_struct); + visitor.visit(create_query); + return std::move(table_struct); } -ASTPtr CreateQueryMatcher::Data::getFormattedPartitionByExpression() +MySQLTableStruct visitCreateQuery(const String & create_query, const Context & context, const std::string & new_database) { - ASTPtr partition_columns = std::make_shared(); + MySQLParser::ParserCreateQuery p_create_query; + ASTPtr ast_create_query = parseQuery(p_create_query, create_query.data(), create_query.data() + create_query.size(), "", 0, 0); - if (!partition_keys.empty()) - partition_columns->children = partition_keys; - else if (!primary_keys.empty()) - { - ASTPtr expr_list = std::make_shared(); - expr_list->children = primary_keys; - - auto syntax = SyntaxAnalyzer(context).analyze(expr_list, columns_name_and_type); - auto index_expr = ExpressionAnalyzer(expr_list, syntax, context).getActions(false); - const NamesAndTypesList & required_names_and_types = index_expr->getRequiredColumnsWithTypes(); - - const auto & addPartitionColumn = [&](const String & column_name, const DataTypePtr & type, Field better_pre_range_size) - { - partition_columns->children.emplace_back(std::make_shared(column_name)); - - if (type->isNullable()) - partition_columns->children.back() = makeASTFunction("assumeNotNull", partition_columns->children.back()); - - if (!better_pre_range_size.isNull()) - partition_columns->children.back() - = makeASTFunction("divide", partition_columns->children.back(), std::make_shared(better_pre_range_size)); - }; - - for (const auto & required_name_and_type : required_names_and_types) - { - DataTypePtr assume_not_null = required_name_and_type.type; - if (assume_not_null->isNullable()) - assume_not_null = (static_cast(*assume_not_null)).getNestedType(); - - WhichDataType which(assume_not_null); - if (which.isInt8()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isInt16()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isInt32()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isInt64()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isUInt8()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isUInt16()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isUInt32()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isUInt64()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isDateOrDateTime()) - { - partition_columns->children.emplace_back(std::make_shared(required_name_and_type.name)); - - if (required_name_and_type.type->isNullable()) - partition_columns->children.back() = makeASTFunction("assumeNotNull", partition_columns->children.back()); - - partition_columns->children.back() = makeASTFunction("toYYYYMM", partition_columns->children.back()); - } - } - } - - const auto function = std::make_shared(); - function->name = "tuple"; - function->arguments = partition_columns; - function->children.push_back(function->arguments); - return function; -} + if (!ast_create_query || !ast_create_query->as()) + throw Exception("LOGICAL ERROR: ast cannot cast to MySQLParser::ASTCreateQuery.", ErrorCodes::LOGICAL_ERROR); + return visitCreateQuery(ast_create_query, context, new_database); } } diff --git a/src/Interpreters/MySQL/CreateQueryConvertVisitor.h b/src/Interpreters/MySQL/CreateQueryVisitor.h similarity index 60% rename from src/Interpreters/MySQL/CreateQueryConvertVisitor.h rename to src/Interpreters/MySQL/CreateQueryVisitor.h index 10fcb0e7b7e..2a57ab817c1 100644 --- a/src/Interpreters/MySQL/CreateQueryConvertVisitor.h +++ b/src/Interpreters/MySQL/CreateQueryVisitor.h @@ -2,15 +2,34 @@ #include #include -#include #include -#include +#include #include +#include #include +#include namespace DB { +struct MySQLTableStruct +{ + bool if_not_exists; + String table_name; + String database_name; + ASTs primary_keys; + ASTs partition_keys; + NamesAndTypesList columns_name_and_type; + + MySQLTableStruct() {} + + MySQLTableStruct(const ASTs & primary_keys_, const ASTs & partition_keys_, const NamesAndTypesList & columns_name_and_type_) + : primary_keys(primary_keys_), partition_keys(partition_keys_), columns_name_and_type(columns_name_and_type_) + {} + + bool operator==(const MySQLTableStruct & other) const; +}; + namespace MySQLVisitor { @@ -20,25 +39,17 @@ class CreateQueryMatcher public: using Visitor = InDepthNodeVisitor; - struct Data + struct Data : public MySQLTableStruct { - /// SETTINGS - WriteBuffer & out; const Context & context; size_t max_ranges; size_t min_rows_pre_range; - ASTs primary_keys; - ASTs partition_keys; - NamesAndTypesList columns_name_and_type; + Data(const Context & context_) : MySQLTableStruct(), context(context_) {} void addPrimaryKey(const ASTPtr & primary_key); void addPartitionKey(const ASTPtr & partition_key); - - ASTPtr getFormattedOrderByExpression(); - - ASTPtr getFormattedPartitionByExpression(); }; static void visit(ASTPtr & ast, Data & data); @@ -56,10 +67,12 @@ private: static void visit(const MySQLParser::ASTDeclarePartitionOptions & declare_partition_options, const ASTPtr &, Data & data); }; -using CreateQueryConvertVisitor = CreateQueryMatcher::Visitor; +using CreateQueryVisitor = CreateQueryMatcher::Visitor; } +MySQLTableStruct visitCreateQuery(ASTPtr & create_query, const Context & context, const std::string & new_database); +MySQLTableStruct visitCreateQuery(const String & create_query, const Context & context, const std::string & new_database); } diff --git a/src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp b/src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp index beed50b1450..d44c513fea6 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp @@ -1,150 +1,187 @@ -#include -#include +#include #include -#include #include -#include +#include +#include +#include +#include +#include using namespace DB; using namespace MySQLParser; using namespace MySQLVisitor; +static DataTypePtr getType(const String & data_type) +{ + return DataTypeFactory::instance().get(data_type); +} + static ContextShared * contextShared() { static SharedContextHolder shared = Context::createShared(); return shared.get(); } -static String convert(const String & input) +static MySQLTableStruct visitQuery(const String & query) { ParserCreateQuery p_create_query; - ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); + ASTPtr ast = parseQuery(p_create_query, query.data(), query.data() + query.size(), "", 0, 0); - WriteBufferFromOwnString out; - CreateQueryConvertVisitor::Data data{ - .out = out, .context = Context::createGlobal(contextShared()), .max_ranges = 1000, .min_rows_pre_range = 1000000}; - CreateQueryConvertVisitor visitor(data); + CreateQueryVisitor::Data data(Context::createGlobal(contextShared())); + data.max_ranges = 1000; + data.min_rows_pre_range = 1000000; + CreateQueryVisitor visitor(data); visitor.visit(ast); - return out.str(); + return std::move(data); } -TEST(CreateQueryConvert, TestConvertNumberColumnsType) +TEST(CreateQueryVisitor, TestWithNumberColumnsType) { EXPECT_EQ( - convert("CREATE TABLE test(a tinyint, b SMALLINT, c MEDIUMINT, d INT, e INTEGER, f BIGINT, g DECIMAL, h DEC, i NUMERIC, j FIXED, k " - "FLOAT, l DOUBLE, m DOUBLE PRECISION, n REAL)"), - "CREATE TABLE test(`a` Nullable(Int8), `b` Nullable(Int16), `c` Nullable(Int32), `d` Nullable(Int32), `e` Nullable(Int32), `f` " - "Nullable(Int64), `g` Nullable(Decimal(10, 0)), `h` Nullable(Decimal(10, 0)), `i` Nullable(Decimal(10, 0)), `j` " - "Nullable(Decimal(10, 0)), `k` Nullable(Float32), `l` Nullable(Float64), `m` Nullable(Float64), `n` Nullable(Float64)) ENGINE = " - "MergeTree() PARTITION BY tuple() ORDER BY tuple()"); + visitQuery("CREATE TABLE test(a tinyint, b SMALLINT, c MEDIUMINT, d INT, e INTEGER, f BIGINT, g DECIMAL, h DEC, i NUMERIC, j " + "FIXED, k FLOAT, l DOUBLE, m DOUBLE PRECISION, n REAL)"), + MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Nullable(Int8)")}, {"b", getType("Nullable(Int16)")} + , {"c", getType("Nullable(Int32)")}, {"d", getType("Nullable(Int32)")}, {"e", getType("Nullable(Int32)")} + , {"f", getType("Nullable(Int64)")}, {"g", getType("Nullable(Decimal(10, 0))")}, {"h", getType("Nullable(Decimal(10, 0))")} + , {"i", getType("Nullable(Decimal(10, 0))")}, {"j", getType("Nullable(Decimal(10, 0))")} + , {"k", getType("Nullable(Float32)")}, {"l", getType("Nullable(Float64)")}, {"m", getType("Nullable(Float64)")} + , {"n", getType("Nullable(Float64)")}} + ) + ); EXPECT_EQ( - convert("CREATE TABLE test(a tinyint(1), b SMALLINT(1), c MEDIUMINT(1), d INT(1), e INTEGER(1), f BIGINT(1), g DECIMAL(1), h " - "DEC(2, 1), i NUMERIC(4, 3), j FIXED(6, 5), k FLOAT(1), l DOUBLE(1, 2), m DOUBLE PRECISION(3, 4), n REAL(5, 6))"), - "CREATE TABLE test(`a` Nullable(Int8), `b` Nullable(Int16), `c` Nullable(Int32), `d` Nullable(Int32), `e` Nullable(Int32), `f` " - "Nullable(Int64), `g` Nullable(Decimal(1, 0)), `h` Nullable(Decimal(2, 1)), `i` Nullable(Decimal(4, 3)), `j` Nullable(Decimal(6, " - "5)), `k` Nullable(Float32), `l` Nullable(Float64), `m` Nullable(Float64), `n` Nullable(Float64)) ENGINE = MergeTree() PARTITION " - "BY tuple() ORDER BY tuple()"); + visitQuery("CREATE TABLE test(a tinyint(1), b SMALLINT(1), c MEDIUMINT(1), d INT(1), e INTEGER(1), f BIGINT(1), g DECIMAL(1), h " + "DEC(2, 1), i NUMERIC(4, 3), j FIXED(6, 5), k FLOAT(1), l DOUBLE(1, 2), m DOUBLE PRECISION(3, 4), n REAL(5, 6))"), + MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Nullable(Int8)")}, {"b", getType("Nullable(Int16)")} + , {"c", getType("Nullable(Int32)")}, {"d", getType("Nullable(Int32)")}, {"e", getType("Nullable(Int32)")} + , {"f", getType("Nullable(Int64)")}, {"g", getType("Nullable(Decimal(1, 0))")}, {"h", getType("Nullable(Decimal(2, 1))")} + , {"i", getType("Nullable(Decimal(4, 3))")}, {"j", getType("Nullable(Decimal(6, 5))")} + , {"k", getType("Nullable(Float32)")}, {"l", getType("Nullable(Float64)")}, {"m", getType("Nullable(Float64)")} + , {"n", getType("Nullable(Float64)")}} + ) + ); /// UNSIGNED EXPECT_EQ( - convert("CREATE TABLE test(a tinyint UNSIGNED, b SMALLINT(1) UNSIGNED, c MEDIUMINT(1) UNSIGNED, d INT(1) UNSIGNED, e INTEGER(1), f " - "BIGINT(1) UNSIGNED, g DECIMAL(1) UNSIGNED, h DEC(2, 1) UNSIGNED, i NUMERIC(4, 3) UNSIGNED, j FIXED(6, 5) UNSIGNED, k FLOAT(1) " - "UNSIGNED, l DOUBLE(1, 2) UNSIGNED, m DOUBLE PRECISION(3, 4) UNSIGNED, n REAL(5, 6) UNSIGNED)"), - "CREATE TABLE test(`a` Nullable(UInt8), `b` Nullable(UInt16), `c` Nullable(UInt32), `d` Nullable(UInt32), `e` Nullable(Int32), `f` " - "Nullable(UInt64), `g` Nullable(Decimal(1, 0)), `h` Nullable(Decimal(2, 1)), `i` Nullable(Decimal(4, 3)), `j` Nullable(Decimal(6, " - "5)), `k` Nullable(Float32), `l` Nullable(Float64), `m` Nullable(Float64), `n` Nullable(Float64)) ENGINE = MergeTree() PARTITION " - "BY tuple() ORDER BY tuple()"); + visitQuery("CREATE TABLE test(a tinyint UNSIGNED, b SMALLINT(1) UNSIGNED, c MEDIUMINT(1) UNSIGNED, d INT(1) UNSIGNED, e INTEGER(1), f " + "BIGINT(1) UNSIGNED, g DECIMAL(1) UNSIGNED, h DEC(2, 1) UNSIGNED, i NUMERIC(4, 3) UNSIGNED, j FIXED(6, 5) UNSIGNED, k FLOAT(1) " + "UNSIGNED, l DOUBLE(1, 2) UNSIGNED, m DOUBLE PRECISION(3, 4) UNSIGNED, n REAL(5, 6) UNSIGNED)"), + MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Nullable(UInt8)")}, {"b", getType("Nullable(UInt16)")} + , {"c", getType("Nullable(UInt32)")}, {"d", getType("Nullable(UInt32)")}, {"e", getType("Nullable(Int32)")} + , {"f", getType("Nullable(UInt64)")}, {"g", getType("Nullable(Decimal(1, 0))")}, {"h", getType("Nullable(Decimal(2, 1))")} + , {"i", getType("Nullable(Decimal(4, 3))")}, {"j", getType("Nullable(Decimal(6, 5))")} + , {"k", getType("Nullable(Float32)")}, {"l", getType("Nullable(Float64)")}, {"m", getType("Nullable(Float64)")} + , {"n", getType("Nullable(Float64)")}} + ) + ); /// NOT NULL EXPECT_EQ( - convert("CREATE TABLE test(a tinyint NOT NULL, b SMALLINT(1) NOT NULL, c MEDIUMINT(1) NOT NULL, d INT(1) NOT NULL, e INTEGER(1), f " - "BIGINT(1) NOT NULL, g DECIMAL(1) NOT NULL, h DEC(2, 1) NOT NULL, i NUMERIC(4, 3) NOT NULL, j FIXED(6, 5) NOT NULL, k FLOAT(1) NOT " - "NULL, l DOUBLE(1, 2) NOT NULL, m DOUBLE PRECISION(3, 4) NOT NULL, n REAL(5, 6) NOT NULL)"), - "CREATE TABLE test(`a` Int8, `b` Int16, `c` Int32, `d` Int32, `e` Nullable(Int32), `f` Int64, `g` Decimal(1, 0), `h` Decimal(2, " - "1), `i` Decimal(4, 3), `j` Decimal(6, 5), `k` Float32, `l` Float64, `m` Float64, `n` Float64) ENGINE = MergeTree() PARTITION BY " - "tuple() ORDER BY tuple()"); + visitQuery("CREATE TABLE test(a tinyint NOT NULL, b SMALLINT(1) NOT NULL, c MEDIUMINT(1) NOT NULL, d INT(1) NOT NULL, e INTEGER(1), f " + "BIGINT(1) NOT NULL, g DECIMAL(1) NOT NULL, h DEC(2, 1) NOT NULL, i NUMERIC(4, 3) NOT NULL, j FIXED(6, 5) NOT NULL, k FLOAT(1) NOT " + "NULL, l DOUBLE(1, 2) NOT NULL, m DOUBLE PRECISION(3, 4) NOT NULL, n REAL(5, 6) NOT NULL)"), + MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Int8")}, {"b", getType("Int16")} + , {"c", getType("Int32")}, {"d", getType("Int32")}, {"e", getType("Nullable(Int32)")} + , {"f", getType("Int64")}, {"g", getType("Decimal(1, 0)")}, {"h", getType("Decimal(2, 1)")} + , {"i", getType("Decimal(4, 3)")}, {"j", getType("Decimal(6, 5)")} + , {"k", getType("Float32")}, {"l", getType("Float64")}, {"m", getType("Float64")}, {"n", getType("Float64")}} + ) + ); /// ZEROFILL EXPECT_EQ( - convert("CREATE TABLE test(a tinyint ZEROFILL, b SMALLINT(1) ZEROFILL, c MEDIUMINT(1) ZEROFILL, d INT(1) ZEROFILL, e INTEGER(1), f " - "BIGINT(1) ZEROFILL, g DECIMAL(1) ZEROFILL, h DEC(2, 1) ZEROFILL, i NUMERIC(4, 3) ZEROFILL, j FIXED(6, 5) ZEROFILL, k FLOAT(1) " - "ZEROFILL, l DOUBLE(1, 2) ZEROFILL, m DOUBLE PRECISION(3, 4) ZEROFILL, n REAL(5, 6) ZEROFILL)"), - "CREATE TABLE test(`a` Nullable(UInt8), `b` Nullable(UInt16), `c` Nullable(UInt32), `d` Nullable(UInt32), `e` Nullable(Int32), `f` " - "Nullable(UInt64), `g` Nullable(Decimal(1, 0)), `h` Nullable(Decimal(2, 1)), `i` Nullable(Decimal(4, 3)), `j` Nullable(Decimal(6, " - "5)), `k` Nullable(Float32), `l` Nullable(Float64), `m` Nullable(Float64), `n` Nullable(Float64)) ENGINE = MergeTree() PARTITION " - "BY tuple() ORDER BY tuple()"); + visitQuery("CREATE TABLE test(a tinyint ZEROFILL, b SMALLINT(1) ZEROFILL, c MEDIUMINT(1) ZEROFILL, d INT(1) ZEROFILL, e INTEGER(1), f " + "BIGINT(1) ZEROFILL, g DECIMAL(1) ZEROFILL, h DEC(2, 1) ZEROFILL, i NUMERIC(4, 3) ZEROFILL, j FIXED(6, 5) ZEROFILL, k FLOAT(1) " + "ZEROFILL, l DOUBLE(1, 2) ZEROFILL, m DOUBLE PRECISION(3, 4) ZEROFILL, n REAL(5, 6) ZEROFILL)"), + MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Nullable(UInt8)")}, {"b", getType("Nullable(UInt16)")} + , {"c", getType("Nullable(UInt32)")}, {"d", getType("Nullable(UInt32)")}, {"e", getType("Nullable(Int32)")} + , {"f", getType("Nullable(UInt64)")}, {"g", getType("Nullable(Decimal(1, 0))")}, {"h", getType("Nullable(Decimal(2, 1))")} + , {"i", getType("Nullable(Decimal(4, 3))")}, {"j", getType("Nullable(Decimal(6, 5))")} + , {"k", getType("Nullable(Float32)")}, {"l", getType("Nullable(Float64)")}, {"m", getType("Nullable(Float64)")} + , {"n", getType("Nullable(Float64)")}} + ) + ); } -TEST(CreateQueryConvert, TestConvertDateTimesColumnsType) +TEST(CreateQueryVisitor, TestWithDateTimesColumnsType) { EXPECT_EQ( - convert("CREATE TABLE test(a DATE, b DATETIME, c TIMESTAMP, d TIME, e year)"), - "CREATE TABLE test(`a` Nullable(Date), `b` Nullable(DateTime), `c` Nullable(DateTime), `d` Nullable(DateTime64(3)), `e` " - "Nullable(Int16)) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY tuple()"); + visitQuery("CREATE TABLE test(a DATE, b DATETIME, c TIMESTAMP, d TIME, e year)"), + MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Nullable(Date)")}, {"b", getType("Nullable(DateTime)")} + , {"c", getType("Nullable(DateTime)")}, {"d", getType("Nullable(DateTime64(3))")}, {"e", getType("Nullable(Int16)")} } + ) + ); EXPECT_EQ( - convert("CREATE TABLE test(a DATE, b DATETIME(1), c TIMESTAMP(1), d TIME(1), e year(4))"), - "CREATE TABLE test(`a` Nullable(Date), `b` Nullable(DateTime), `c` Nullable(DateTime), `d` Nullable(DateTime64(3)), `e` " - "Nullable(Int16)) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY tuple()"); + visitQuery("CREATE TABLE test(a DATE, b DATETIME(1), c TIMESTAMP(1), d TIME(1), e year(4))"), + MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Nullable(Date)")}, {"b", getType("Nullable(DateTime)")} + , {"c", getType("Nullable(DateTime)")}, {"d", getType("Nullable(DateTime64(3))")}, {"e", getType("Nullable(Int16)")} } + ) + ); EXPECT_EQ( - convert( - "CREATE TABLE test(a DATE NOT NULL, b DATETIME(1) NOT NULL, c TIMESTAMP(1) NOT NULL, d TIME(1) NOT NULL, e year(4) NOT NULL)"), - "CREATE TABLE test(`a` Date, `b` DateTime, `c` DateTime, `d` DateTime64(3), `e` Int16) ENGINE = MergeTree() PARTITION BY tuple() " - "ORDER BY tuple()"); + visitQuery("CREATE TABLE test(a DATE NOT NULL, b DATETIME(1) NOT NULL, c TIMESTAMP(1) NOT NULL, d TIME(1) NOT NULL, e year(4) NOT NULL)"), + MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Date")}, {"b", getType("DateTime")} , {"c", getType("DateTime")}, {"d", getType("DateTime64")}, {"e", getType("Int16")} } + ) + ); } -TEST(CreateQueryConvert, TestConvertParitionOptions) +TEST(CreateQueryVisitor, TestWithParitionOptions) { EXPECT_EQ( - convert("CREATE TABLE test(a DATE NOT NULL) PARTITION BY HASH a"), - "CREATE TABLE test(`a` Date) ENGINE = MergeTree() PARTITION BY tuple(a) ORDER BY tuple()"); + visitQuery("CREATE TABLE test(a DATE NOT NULL) PARTITION BY HASH a"), + MySQLTableStruct(ASTs{}, ASTs{std::make_shared("a")}, {{"a", getType("Date")}})); EXPECT_EQ( - convert("CREATE TABLE test(a DATE NOT NULL) PARTITION BY LINEAR HASH a"), - "CREATE TABLE test(`a` Date) ENGINE = MergeTree() PARTITION BY tuple(a) ORDER BY tuple()"); + visitQuery("CREATE TABLE test(a DATE NOT NULL) PARTITION BY LINEAR HASH a"), + MySQLTableStruct(ASTs{}, ASTs{std::make_shared("a")}, {{"a", getType("Date")}})); EXPECT_EQ( - convert("CREATE TABLE test(a DATE NOT NULL) PARTITION BY RANGE(a)"), - "CREATE TABLE test(`a` Date) ENGINE = MergeTree() PARTITION BY tuple(a) ORDER BY tuple()"); + visitQuery("CREATE TABLE test(a DATE NOT NULL) PARTITION BY RANGE(a)"), + MySQLTableStruct(ASTs{}, ASTs{std::make_shared("a")}, {{"a", getType("Date")}})); EXPECT_EQ( - convert("CREATE TABLE test(a DATE NOT NULL, b INT) PARTITION BY RANGE COLUMNS(a, b)"), - "CREATE TABLE test(`a` Date, `b` Nullable(Int32)) ENGINE = MergeTree() PARTITION BY (a, b) ORDER BY tuple()"); + visitQuery("CREATE TABLE test(a DATE NOT NULL, b INT) PARTITION BY RANGE COLUMNS(a, b)"), + MySQLTableStruct(ASTs{}, ASTs{std::make_shared("a"), std::make_shared("b")}, {{"a", getType("Date")}, {"b", getType("Nullable(Int32)")}})); EXPECT_EQ( - convert("CREATE TABLE test(a DATE NOT NULL) PARTITION BY LIST(a)"), - "CREATE TABLE test(`a` Date) ENGINE = MergeTree() PARTITION BY tuple(a) ORDER BY tuple()"); + visitQuery("CREATE TABLE test(a DATE NOT NULL) PARTITION BY LIST(a)"), + MySQLTableStruct(ASTs{}, ASTs{std::make_shared("a")}, {{"a", getType("Date")}})); EXPECT_EQ( - convert("CREATE TABLE test(a DATE NOT NULL, b INT) PARTITION BY LIST COLUMNS(a, b)"), - "CREATE TABLE test(`a` Date, `b` Nullable(Int32)) ENGINE = MergeTree() PARTITION BY (a, b) ORDER BY tuple()"); + visitQuery("CREATE TABLE test(a DATE NOT NULL, b INT) PARTITION BY LIST COLUMNS(a, b)"), + MySQLTableStruct(ASTs{}, ASTs{std::make_shared("a"), std::make_shared("b")}, + {{"a", getType("Date")}, {"b", getType("Nullable(Int32)")}})); } -TEST(CreateQueryConvert, TestConvertPrimaryToPartitionBy) +TEST(CreateQueryVisitor, TestWithPrimaryToPartitionBy) { - EXPECT_EQ(convert("CREATE TABLE test(a DATE NOT NULL PRIMARY KEY)"), - "CREATE TABLE test(`a` Date) ENGINE = MergeTree() PARTITION BY tuple(toYYYYMM(a)) ORDER BY tuple(a)"); - - EXPECT_EQ(convert("CREATE TABLE test(a DATETIME NOT NULL PRIMARY KEY)"), - "CREATE TABLE test(`a` DateTime) ENGINE = MergeTree() PARTITION BY tuple(toYYYYMM(a)) ORDER BY tuple(a)"); - - EXPECT_EQ(convert("CREATE TABLE test(a TINYINT NOT NULL PRIMARY KEY)"), - "CREATE TABLE test(`a` Int8) ENGINE = MergeTree() PARTITION BY tuple(a) ORDER BY tuple(a)"); - - EXPECT_EQ(convert("CREATE TABLE test(a SMALLINT NOT NULL PRIMARY KEY)"), - "CREATE TABLE test(`a` Int16) ENGINE = MergeTree() PARTITION BY tuple(a) ORDER BY tuple(a)"); - - EXPECT_EQ(convert("CREATE TABLE test(a INT NOT NULL PRIMARY KEY)"), - "CREATE TABLE test(`a` Int32) ENGINE = MergeTree() PARTITION BY tuple(a / 4294968) ORDER BY tuple(a)"); - - EXPECT_EQ(convert("CREATE TABLE test(a BIGINT NOT NULL PRIMARY KEY)"), - "CREATE TABLE test(`a` Int64) ENGINE = MergeTree() PARTITION BY tuple(a / 18446744073709552) ORDER BY tuple(a)"); + EXPECT_EQ( + visitQuery("CREATE TABLE test(a DATE NOT NULL PRIMARY KEY)"), + MySQLTableStruct(ASTs{std::make_shared("a")}, ASTs{}, {{"a", getType("Date")}})); EXPECT_EQ( - convert("CREATE TABLE test(a BIGINT PRIMARY KEY)"), - "CREATE TABLE test(`a` Nullable(Int64)) ENGINE = MergeTree() PARTITION BY tuple(assumeNotNull(a) / 18446744073709552) ORDER BY " - "tuple(a)"); + visitQuery("CREATE TABLE test(a DATETIME NOT NULL PRIMARY KEY)"), + MySQLTableStruct(ASTs{std::make_shared("a")}, ASTs{}, {{"a", getType("DateTime")}})); + + EXPECT_EQ( + visitQuery("CREATE TABLE test(a TINYINT NOT NULL PRIMARY KEY)"), + MySQLTableStruct(ASTs{std::make_shared("a")}, ASTs{}, {{"a", getType("Int8")}})); + + EXPECT_EQ( + visitQuery("CREATE TABLE test(a SMALLINT NOT NULL PRIMARY KEY)"), + MySQLTableStruct(ASTs{std::make_shared("a")}, ASTs{}, {{"a", getType("Int16")}})); + + EXPECT_EQ( + visitQuery("CREATE TABLE test(a INT NOT NULL PRIMARY KEY)"), + MySQLTableStruct(ASTs{std::make_shared("a")}, ASTs{}, {{"a", getType("Int32")}})); + + EXPECT_EQ( + visitQuery("CREATE TABLE test(a BIGINT NOT NULL PRIMARY KEY)"), + MySQLTableStruct(ASTs{std::make_shared("a")}, ASTs{}, {{"a", getType("Int64")}})); + + EXPECT_EQ( + visitQuery("CREATE TABLE test(a BIGINT PRIMARY KEY)"), + MySQLTableStruct(ASTs{std::make_shared("a")}, ASTs{}, {{"a", getType("Nullable(Int64)")}})); } From 0336a4ad58037bdb92fda6a5d141e50855253dc3 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 12 Jun 2020 12:21:43 +0800 Subject: [PATCH 067/374] ISSUES-4006 suport synchronous MySQL incremental data[part 2] --- src/Core/MySQLReplication.h | 1 + .../AddingVersionsBlockOutputStream.cpp | 13 +- src/DataTypes/DataTypeEnum.h | 3 +- src/Databases/DatabaseFactory.cpp | 13 +- src/Databases/MySQL/DataBuffers.cpp | 135 --------- src/Databases/MySQL/DataBuffers.h | 42 --- .../MySQL/DatabaseMaterializeMySQL.cpp | 225 +++++++-------- .../MySQL/DatabaseMaterializeMySQL.h | 137 ++------- .../MySQL/DatabaseMaterializeMySQLWrap.cpp | 188 +++++++++++++ .../MySQL/DatabaseMaterializeMySQLWrap.h | 70 +++++ .../MySQL/DatabaseMaterializeTablesIterator.h | 34 +++ src/Databases/MySQL/EventConsumer.cpp | 261 ++++++++++++++++++ src/Databases/MySQL/EventConsumer.h | 63 +++++ ...StatusInfo.cpp => MaterializeMetadata.cpp} | 111 +++++--- ...sterStatusInfo.h => MaterializeMetadata.h} | 11 +- .../MySQL/MaterializeModeSettings.cpp | 42 +++ src/Databases/MySQL/MaterializeModeSettings.h | 28 ++ src/Databases/MySQL/queryConvert.cpp | 13 +- src/Parsers/ParserSetQuery.cpp | 6 +- src/Storages/StorageMaterializeMySQL.cpp | 56 ++++ src/Storages/StorageMaterializeMySQL.h | 28 ++ 21 files changed, 996 insertions(+), 484 deletions(-) delete mode 100644 src/Databases/MySQL/DataBuffers.cpp delete mode 100644 src/Databases/MySQL/DataBuffers.h create mode 100644 src/Databases/MySQL/DatabaseMaterializeMySQLWrap.cpp create mode 100644 src/Databases/MySQL/DatabaseMaterializeMySQLWrap.h create mode 100644 src/Databases/MySQL/DatabaseMaterializeTablesIterator.h create mode 100644 src/Databases/MySQL/EventConsumer.cpp create mode 100644 src/Databases/MySQL/EventConsumer.h rename src/Databases/MySQL/{MasterStatusInfo.cpp => MaterializeMetadata.cpp} (52%) rename src/Databases/MySQL/{MasterStatusInfo.h => MaterializeMetadata.h} (72%) create mode 100644 src/Databases/MySQL/MaterializeModeSettings.cpp create mode 100644 src/Databases/MySQL/MaterializeModeSettings.h create mode 100644 src/Storages/StorageMaterializeMySQL.cpp create mode 100644 src/Storages/StorageMaterializeMySQL.h diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index 420bd8fdc02..f4ac02c2ffd 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -460,6 +460,7 @@ namespace MySQLReplication String binlog_name; Position() : binlog_pos(0), binlog_name("") { } + Position(UInt64 binlog_pos_, const String & binlog_name_) : binlog_pos(binlog_pos_), binlog_name(binlog_name_) { } void updateLogPos(UInt64 pos) { binlog_pos = pos; } void updateLogName(String binlog) { binlog_name = std::move(binlog); } }; diff --git a/src/DataStreams/AddingVersionsBlockOutputStream.cpp b/src/DataStreams/AddingVersionsBlockOutputStream.cpp index 2adc1488e98..c2c83e6adc9 100644 --- a/src/DataStreams/AddingVersionsBlockOutputStream.cpp +++ b/src/DataStreams/AddingVersionsBlockOutputStream.cpp @@ -23,22 +23,21 @@ void AddingVersionsBlockOutputStream::flush() void AddingVersionsBlockOutputStream::write(const Block & block) { - /// create_version and delete_version are always in the last place Block res; size_t rows = block.rows(); for (size_t index = 0; index < block.columns(); ++index) res.insert(block.getByPosition(index)); - DataTypePtr data_type = std::make_shared(); + DataTypePtr sign_type = std::make_shared(); + DataTypePtr version_type = std::make_shared(); - auto create_version = ColumnUInt64::create(rows); - for (size_t index = 0; index < rows; ++index) - create_version->getData()[index] = ++version; + ColumnPtr sign_column = sign_type->createColumnConst(rows, Field(Int8(1)))->convertToFullColumnIfConst(); + ColumnPtr version_column = version_type->createColumnConst(rows, Field(UInt64(++version)))->convertToFullColumnIfConst(); Block header = output->getHeader(); - res.insert(ColumnWithTypeAndName(create_version->getPtr(), data_type, header.getByPosition(header.columns() - 2).name)); - res.insert(ColumnWithTypeAndName(data_type->createColumnConstWithDefaultValue(rows)->convertToFullColumnIfConst(), data_type, header.getByPosition(header.columns() - 1).name)); + res.insert({sign_column, sign_type, header.getByPosition(header.columns() - 2).name}); + res.insert({version_column, version_type, header.getByPosition(header.columns() - 1).name}); output->write(res); } Block AddingVersionsBlockOutputStream::getHeader() const diff --git a/src/DataTypes/DataTypeEnum.h b/src/DataTypes/DataTypeEnum.h index 80b41692cdd..4756a218778 100644 --- a/src/DataTypes/DataTypeEnum.h +++ b/src/DataTypes/DataTypeEnum.h @@ -43,7 +43,8 @@ public: using ColumnType = ColumnVector; using Value = std::pair; using Values = std::vector; - using NameToValueMap = HashMap; + using NameToValueMap = HashMap; using ValueToNameMap = std::unordered_map; static constexpr bool is_parametric = true; diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 87a8e6d775a..e7f8b6f22de 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -18,12 +18,13 @@ #endif #if USE_MYSQL +# include # include # include +# include # include # include # include -# include #endif namespace DB @@ -118,12 +119,18 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String const auto & [remote_host_name, remote_port] = parseAddress(host_name_and_port, 3306); auto mysql_pool = mysqlxx::Pool(mysql_database_name, remote_host_name, mysql_user_name, mysql_user_password, remote_port); - if (materializeMySQLDatabase(engine_define->settings)) + auto materialize_mode_settings = std::make_unique(); + + if (engine_define->settings) + materialize_mode_settings->loadFromQuery(*engine_define); + + if (materialize_mode_settings->locality_data) { MySQLClient client(remote_host_name, remote_port, mysql_user_name, mysql_user_password); return std::make_shared( - context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_pool), std::move(client)); + context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_pool), std::move(client) + , std::move(materialize_mode_settings)); } return std::make_shared(context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_pool)); diff --git a/src/Databases/MySQL/DataBuffers.cpp b/src/Databases/MySQL/DataBuffers.cpp deleted file mode 100644 index 8659eadb8bb..00000000000 --- a/src/Databases/MySQL/DataBuffers.cpp +++ /dev/null @@ -1,135 +0,0 @@ -#include - -#include -#include - -namespace DB -{ - -DataBuffers::DataBuffers(size_t & version_, IDatabase * database_, const std::function &)> & flush_function_) - : version(version_), database(database_), flush_function(flush_function_) -{ - - /// TODO: 定时刷新 -} - -void DataBuffers::flush() -{ - flush_function(buffers); - buffers.clear(); -} - -void DataBuffers::writeData(const std::string & table_name, const std::vector & rows_data) -{ - Block & to = getTableBuffer(table_name, rows_data.size()); - for (size_t column = 0; column < to.columns() - 2; ++column) - { - /// normally columns - MutableColumnPtr col_to = (*std::move(to.getByPosition(column).column)).mutate(); - - for (size_t index = 0; index < rows_data.size(); ++index) - col_to->insert(DB::get(rows_data[index])[column]); - } - - Field new_version(UInt64(++version)); - MutableColumnPtr create_version_column = (*std::move(to.getByPosition(to.columns() - 2)).column).mutate(); - MutableColumnPtr delete_version_column = (*std::move(to.getByPosition(to.columns() - 1)).column).mutate(); - - delete_version_column->insertManyDefaults(rows_data.size()); - - for (size_t index = 0; index < rows_data.size(); ++index) - create_version_column->insert(new_version); -} - -void DataBuffers::updateData(const String & table_name, const std::vector & rows_data) -{ - if (rows_data.size() % 2 != 0) - throw Exception("LOGICAL ERROR: ", ErrorCodes::LOGICAL_ERROR); - - Block & to = getTableBuffer(table_name, rows_data.size()); - for (size_t column = 0; column < to.columns() - 2; ++column) - { - /// normally columns - MutableColumnPtr col_to = (*std::move(to.getByPosition(column).column)).mutate(); - - for (size_t index = 0; index < rows_data.size(); ++index) - col_to->insert(DB::get(rows_data[index])[column]); - } - - Field new_version(UInt64(++version)); - MutableColumnPtr create_version_column = (*std::move(to.getByPosition(to.columns() - 2)).column).mutate(); - MutableColumnPtr delete_version_column = (*std::move(to.getByPosition(to.columns() - 1)).column).mutate(); - - for (size_t index = 0; index < rows_data.size(); ++index) - { - if (index % 2 == 0) - { - create_version_column->insertDefault(); - delete_version_column->insert(new_version); - } - else - { - delete_version_column->insertDefault(); - create_version_column->insert(new_version); - } - } -} - -void DataBuffers::deleteData(const String & table_name, const std::vector & rows_data) -{ - Block & to = getTableBuffer(table_name, rows_data.size()); - for (size_t column = 0; column < to.columns() - 2; ++column) - { - /// normally columns - MutableColumnPtr col_to = (*std::move(to.getByPosition(column).column)).mutate(); - - for (size_t index = 0; index < rows_data.size(); ++index) - col_to->insert(DB::get(rows_data[index])[column]); - } - - Field new_version(UInt64(++version)); - MutableColumnPtr create_version_column = (*std::move(to.getByPosition(to.columns() - 2)).column).mutate(); - MutableColumnPtr delete_version_column = (*std::move(to.getByPosition(to.columns() - 1)).column).mutate(); - - create_version_column->insertManyDefaults(rows_data.size()); - - for (size_t index = 0; index < rows_data.size(); ++index) - delete_version_column->insert(new_version); -} - -Block & DataBuffers::getTableBuffer(const String & table_name, size_t write_size) -{ - if (buffers.find(table_name) == buffers.end()) - { - StoragePtr write_storage = database->tryGetTable(table_name); - buffers[table_name] = write_storage->getSampleBlockNonMaterialized(); - } - - /// TODO: settings - if (buffers[table_name].rows() + write_size > 8192) - flush(); - - return buffers[table_name]; -} - -[[noreturn]] void DataBuffers::scheduleFlush() -{ - while (1) - { - try - { - flush(); - sleepForSeconds(1); - } - catch (...) - { -// ++error_count; -// sleep_time = std::min( -// std::chrono::milliseconds{Int64(default_sleep_time.count() * std::exp2(error_count))}, -// max_sleep_time); - tryLogCurrentException(""); - } - } -} - -} diff --git a/src/Databases/MySQL/DataBuffers.h b/src/Databases/MySQL/DataBuffers.h deleted file mode 100644 index 4e69eb438d7..00000000000 --- a/src/Databases/MySQL/DataBuffers.h +++ /dev/null @@ -1,42 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class DataBuffers : private boost::noncopyable -{ -public: - DataBuffers(size_t & version_, IDatabase * database_, const std::function &)> & flush_function_); - - void flush(); - - void writeData(const std::string & table_name, const std::vector & rows_data); - - void updateData(const std::string & table_name, const std::vector & rows_data); - - void deleteData(const std::string & table_name, const std::vector & rows_data); - - -private: - size_t & version; - IDatabase * database; - std::function &)> flush_function; - - mutable std::mutex mutex; - std::unordered_map buffers; - - [[noreturn]] void scheduleFlush(); - - Block & getTableBuffer(const String & table_name, size_t write_size); - - ThreadFromGlobalPool thread{&DataBuffers::scheduleFlush, this}; -}; - -} diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 63676e74f30..b3d55c46d66 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -10,17 +10,14 @@ # include # include # include -# include -# include -# include -# include # include +# include +# include # include # include # include # include # include -# include # include # include # include @@ -34,189 +31,165 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; } -DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( - const Context & context, const String & database_name_, const String & metadata_path_, - const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_) - : IDatabase(database_name_), global_context(context.getGlobalContext()), metadata_path(metadata_path_) - , database_engine_define(database_engine_define_->clone()), mysql_database_name(mysql_database_name_) - , nested_database(std::make_shared(database_name_, metadata_path, global_context)) - , pool(std::move(pool_)), client(std::move(client_)), log(&Logger::get("DatabaseMaterializeMySQL")) +static inline BlockIO tryToExecuteQuery(const String & query_to_execute, const Context & context_, const String & comment) { - /// TODO: 做简单的check, 失败即报错 -} - -BlockIO DatabaseMaterializeMySQL::tryToExecuteQuery(const String & query_to_execute, const String & comment) -{ - String query_prefix = "/*" + comment + " for " + backQuoteIfNeed(database_name) + " Database */ "; - try { - Context context = global_context; + Context context = context_; context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; context.setCurrentQueryId(""); // generate random query_id - return executeQuery(query_prefix + query_to_execute, context, true); + return executeQuery("/*" + comment + "*/ " + query_to_execute, context, true); } catch (...) { - tryLogCurrentException(log, "Query " + query_to_execute + " wasn't finished successfully"); + tryLogCurrentException("DatabaseMaterializeMySQL", "Query " + query_to_execute + " wasn't finished successfully"); throw; } - LOG_DEBUG(log, "Executed query: " << query_to_execute); + LOG_DEBUG(&Logger::get("DatabaseMaterializeMySQL"), "Executed query: " << query_to_execute); } -String DatabaseMaterializeMySQL::getCreateQuery(const mysqlxx::Pool::Entry & connection, const String & table_name) +DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( + const Context & context, const String & database_name_, const String & metadata_path_ + , const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_ + , MySQLClient && client_ , std::unique_ptr settings_) + : DatabaseMaterializeMySQLWrap(std::make_shared(database_name_, metadata_path_, context), database_engine_define_->clone(), "DatabaseMaterializeMySQL") + , global_context(context.getGlobalContext()), metadata_path(metadata_path_), mysql_database_name(mysql_database_name_) + , pool(std::move(pool_)), client(std::move(client_)), settings(std::move(settings_)) { - Block show_create_table_header{ - {std::make_shared(), "Table"}, - {std::make_shared(), "Create Table"}, - }; - - MySQLBlockInputStream show_create_table( - connection, "SHOW CREATE TABLE " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name), - show_create_table_header, DEFAULT_BLOCK_SIZE); - - Block create_query_block = show_create_table.read(); - if (!create_query_block || create_query_block.rows() != 1) - throw Exception("LOGICAL ERROR mysql show create return more rows.", ErrorCodes::LOGICAL_ERROR); - - return create_query_block.getByName("Create Table").column->getDataAt(0).toString(); + /// TODO: 做简单的check, 失败即报错 + scheduleSynchronized(); } -BlockOutputStreamPtr DatabaseMaterializeMySQL::getTableOutput(const String & table_name, bool fill_version) +BlockOutputStreamPtr DatabaseMaterializeMySQL::getTableOutput(const String & table_name) { - Context context = global_context; - context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - context.setCurrentQueryId(""); // generate random query_id + String with_database_table_name = backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name); + BlockIO res = tryToExecuteQuery("INSERT INTO " + with_database_table_name + " VALUES", global_context, ""); - StoragePtr write_storage = nested_database->tryGetTable(table_name); - auto table_lock = write_storage->lockStructureForShare(true, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); + if (!res.out) + throw Exception("LOGICAL ERROR:", ErrorCodes::LOGICAL_ERROR); - BlockOutputStreamPtr output = write_storage->write(ASTPtr{}, global_context); - output->addTableLock(table_lock); - - return fill_version ? std::make_shared(version, output) : output; + return res.out; } -void DatabaseMaterializeMySQL::dumpDataForTables(mysqlxx::Pool::Entry & connection, const std::vector & tables_name, const std::function & is_cancelled) +void DatabaseMaterializeMySQL::cleanOutdatedTables() { - std::unordered_map tables_create_query; - for (size_t index = 0; index < tables_name.size() && !is_cancelled(); ++index) - tables_create_query[tables_name[index]] = getCreateQuery(connection, tables_name[index]); + auto ddl_guard = DatabaseCatalog::instance().getDDLGuard(database_name, ""); + const DatabasePtr & clean_database = DatabaseCatalog::instance().getDatabase(database_name); - auto iterator = tables_create_query.begin(); - for (; iterator != tables_create_query.end() && !is_cancelled(); ++iterator) + for (auto iterator = clean_database->getTablesIterator(); iterator->isValid(); iterator->next()) + { + String table = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(iterator->name()); + String comment = String("Clean ") + table + " for dump mysql."; + tryToExecuteQuery("DROP TABLE " + table, global_context, comment); + } +} + +void DatabaseMaterializeMySQL::dumpDataForTables(mysqlxx::Pool::Entry & connection, MaterializeMetadata & master_info, const std::function & is_cancelled) +{ + auto iterator = master_info.need_dumping_tables.begin(); + for (; iterator != master_info.need_dumping_tables.end() && !is_cancelled(); ++iterator) { const auto & table_name = iterator->first; MySQLTableStruct table_struct = visitCreateQuery(iterator->second, global_context, database_name); String comment = String("Dumping ") + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name); - tryToExecuteQuery(toCreateQuery(table_struct, global_context), comment); + tryToExecuteQuery(toCreateQuery(table_struct, global_context), global_context, comment); - BlockOutputStreamPtr out = getTableOutput(table_name, true); + BlockOutputStreamPtr out = std::make_shared(master_info.version, getTableOutput(table_name)); MySQLBlockInputStream input(connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name), out->getHeader(), DEFAULT_BLOCK_SIZE); copyData(input, *out, is_cancelled); } } -MasterStatusInfo DatabaseMaterializeMySQL::prepareSynchronized(std::unique_lock & lock) +std::optional DatabaseMaterializeMySQL::prepareSynchronized(std::unique_lock & lock, const std::function & is_cancelled) { - while(!sync_quit.load(std::memory_order_seq_cst)) + while (!is_cancelled()) { try { LOG_DEBUG(log, "Checking " + database_name + " database status."); - while (!sync_quit && !DatabaseCatalog::instance().isDatabaseExist(database_name)) + while (!is_cancelled && !DatabaseCatalog::instance().isDatabaseExist(database_name)) sync_cond.wait_for(lock, std::chrono::seconds(1)); LOG_DEBUG(log, database_name + " database status is OK."); mysqlxx::PoolWithFailover::Entry connection = pool.get(); - MasterStatusInfo master_info(connection, getMetadataPath() + "/.master_status", mysql_database_name); + MaterializeMetadata metadata(connection, getMetadataPath() + "/.metadata", mysql_database_name); - if (!master_info.need_dumping_tables.empty()) + if (!metadata.need_dumping_tables.empty()) { - /// TODO: 删除所有表结构, 这可能需要考虑到仍然有查询在使用这个表. - dumpDataForTables(connection, master_info.need_dumping_tables, [&]() { return sync_quit.load(std::memory_order_seq_cst); }); - master_info.finishDump(); + metadata.transaction(Position(metadata.binlog_position, metadata.binlog_file), [&]() + { + cleanOutdatedTables(); + dumpDataForTables(connection, metadata, is_cancelled); + }); } client.connect(); - client.startBinlogDump(std::rand(), mysql_database_name, master_info.binlog_file, master_info.binlog_position); - return master_info; + client.startBinlogDump(std::rand(), mysql_database_name, metadata.binlog_file, metadata.binlog_position); + return metadata; + } + catch (mysqlxx::Exception & ) + { + tryLogCurrentException(log); + + /// Avoid busy loop when MySQL is not available. + sleepForMilliseconds(settings->max_wait_time_when_mysql_unavailable); + } + } + + return {}; +} + +void DatabaseMaterializeMySQL::scheduleSynchronized() +{ + background_thread_pool.scheduleOrThrowOnError([&]() + { + ThreadStatus thread_status; + setThreadName("MySQLDBSync"); + + std::unique_lock lock(sync_mutex); + const auto quit_requested = [this] { return sync_quit.load(std::memory_order_relaxed); }; + + try + { + std::optional metadata = prepareSynchronized(lock, quit_requested); + + if (!quit_requested() && metadata) + { + EventConsumer consumer(getDatabaseName(), global_context, *metadata, *settings); + + while (!quit_requested()) + { + const auto & event = client.readOneBinlogEvent(); + consumer.onEvent(event, client.getPosition()); + } + } } catch(...) { - tryLogCurrentException(log, "Prepare MySQL Synchronized exception and retry"); - - sleepForSeconds(1); + setException(std::current_exception()); } - } - - throw Exception("", ErrorCodes::LOGICAL_ERROR); + }); } - -void DatabaseMaterializeMySQL::synchronized() +DatabaseMaterializeMySQL::~DatabaseMaterializeMySQL() { - setThreadName("MySQLDBSync"); - try { - std::unique_lock lock{sync_mutex}; - - MasterStatusInfo master_status = prepareSynchronized(lock); - - DataBuffers buffers(version, this, [&](const std::unordered_map & tables_data) + if (!sync_quit) { - master_status.transaction(client.getPosition(), [&]() /// At least once, There is only one possible reference: https://github.com/ClickHouse/ClickHouse/pull/8467 { - for (const auto & [table_name, data] : tables_data) - { - if (!sync_quit.load(std::memory_order_seq_cst)) - { - LOG_DEBUG(log, "Prepare to flush data."); - BlockOutputStreamPtr output = getTableOutput(table_name, false); - output->writePrefix(); - output->write(data); - output->writeSuffix(); - output->flush(); - LOG_DEBUG(log, "Finish data flush."); - } - } - }); - }); + sync_quit = true; + std::lock_guard lock(sync_mutex); + } - while (!sync_quit.load(std::memory_order_seq_cst)) - { - const auto & event = client.readOneBinlogEvent(); - - if (event->type() == MYSQL_WRITE_ROWS_EVENT) - { - WriteRowsEvent & write_rows_event = static_cast(*event); - write_rows_event.dump(); - buffers.writeData(write_rows_event.table, write_rows_event.rows); - } - else if (event->type() == MYSQL_UPDATE_ROWS_EVENT) - { - UpdateRowsEvent & update_rows_event = static_cast(*event); - update_rows_event.dump(); - buffers.updateData(update_rows_event.table, update_rows_event.rows); - } - else if (event->type() == MYSQL_DELETE_ROWS_EVENT) - { - DeleteRowsEvent & delete_rows_event = static_cast(*event); - delete_rows_event.dump(); - buffers.deleteData(delete_rows_event.table, delete_rows_event.rows); - } - else if (event->type() == MYSQL_QUERY_EVENT) - { - /// TODO: 识别, 查看是否支持的DDL, 支持的话立即刷新当前的数据, 然后执行DDL. - buffers.flush(); - } + sync_cond.notify_one(); + background_thread_pool.wait(); } } - catch(...) + catch (...) { - tryLogCurrentException(log); + tryLogCurrentException(__PRETTY_FUNCTION__); } } diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h index b6b27d33cae..74526bb4635 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -1,6 +1,7 @@ #pragma once #include "config_core.h" + #if USE_MYSQL # include @@ -10,7 +11,9 @@ # include # include # include -# include +# include +# include +# include # include # include # include @@ -19,149 +22,41 @@ namespace DB { -class DatabaseMaterializeMySQL : public IDatabase +class DatabaseMaterializeMySQL : public DatabaseMaterializeMySQLWrap { public: + ~DatabaseMaterializeMySQL() override; + DatabaseMaterializeMySQL( const Context & context, const String & database_name_, const String & metadata_path_, const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_, - MySQLClient && client_); + MySQLClient && client_, std::unique_ptr settings_); String getEngineName() const override { return "MySQL"; } - void shutdown() override { nested_database->shutdown(); } - - bool empty() const override { return nested_database->empty(); } - - String getDataPath() const override { return nested_database->getDataPath(); } - - String getTableDataPath(const String & string) const override { return nested_database->getTableDataPath(string); } - - String getTableDataPath(const ASTCreateQuery & query) const override { return nested_database->getTableDataPath(query); } - - UUID tryGetTableUUID(const String & string) const override { return nested_database->tryGetTableUUID(string); } - - bool isDictionaryExist(const String & string) const override { return nested_database->isDictionaryExist(string); } - - DatabaseDictionariesIteratorPtr getDictionariesIterator(const FilterByNameFunction & filter_by_dictionary_name) override - { - return nested_database->getDictionariesIterator(filter_by_dictionary_name); - } - - void createTable(const Context & context, const String & string, const StoragePtr & ptr, const ASTPtr & astPtr) override - { - nested_database->createTable(context, string, ptr, astPtr); - } - - void createDictionary(const Context & context, const String & string, const ASTPtr & ptr) override - { - nested_database->createDictionary(context, string, ptr); - } - - void dropTable(const Context & context, const String & string, bool no_delay) override - { - nested_database->dropTable(context, string, no_delay); - } - - void removeDictionary(const Context & context, const String & string) override { nested_database->removeDictionary(context, string); } - - void attachTable(const String & string, const StoragePtr & ptr, const String & relative_table_path) override - { - nested_database->attachTable(string, ptr, relative_table_path); - } - - void attachDictionary(const String & string, const DictionaryAttachInfo & info) override { nested_database->attachDictionary(string, info); } - - StoragePtr detachTable(const String & string) override { return nested_database->detachTable(string); } - - void detachDictionary(const String & string) override { nested_database->detachDictionary(string); } - - void renameTable(const Context & context, const String & string, IDatabase & database, const String & string1, bool b) override - { - nested_database->renameTable(context, string, database, string1, b); - } - - void alterTable(const Context & context, const StorageID & id, const StorageInMemoryMetadata & metadata) override - { - nested_database->alterTable(context, id, metadata); - } - - time_t getObjectMetadataModificationTime(const String & string) const override - { - return nested_database->getObjectMetadataModificationTime(string); - } - - Poco::AutoPtr getDictionaryConfiguration(const String & string) const override - { - return nested_database->getDictionaryConfiguration(string); - } - - String getMetadataPath() const override { return nested_database->getMetadataPath(); } - - String getObjectMetadataPath(const String & string) const override { return nested_database->getObjectMetadataPath(string); } - - bool shouldBeEmptyOnDetach() const override { return nested_database->shouldBeEmptyOnDetach(); } - - void drop(const Context & context) override { nested_database->drop(context); } - - bool isTableExist(const String & name) const override { return nested_database->isTableExist(name); } - - StoragePtr tryGetTable(const String & name) const override { return nested_database->tryGetTable(name); } - - void loadStoredObjects(Context & context, bool b) override - { - try - { - LOG_DEBUG(log, "Loading MySQL nested database stored objects."); - nested_database->loadStoredObjects(context, b); - LOG_DEBUG(log, "Loaded MySQL nested database stored objects."); - } - catch (...) - { - tryLogCurrentException(log, "Cannot load MySQL nested database stored objects."); - throw; - } - } - - ASTPtr getCreateDatabaseQuery() const override - { - const auto & create_query = std::make_shared(); - create_query->database = database_name; - create_query->set(create_query->storage, database_engine_define); - return create_query; - } - - DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override { return nested_database->getTablesIterator(filter_by_table_name); } - private: const Context & global_context; String metadata_path; - ASTPtr database_engine_define; String mysql_database_name; - DatabasePtr nested_database; - size_t version{0}; mutable mysqlxx::Pool pool; mutable MySQLClient client; + std::unique_ptr settings; - Poco::Logger * log; + void cleanOutdatedTables(); - void synchronized(); + void scheduleSynchronized(); - MasterStatusInfo prepareSynchronized(std::unique_lock & lock); + BlockOutputStreamPtr getTableOutput(const String & table_name); - BlockOutputStreamPtr getTableOutput(const String & table_name, bool fill_version); + std::optional prepareSynchronized(std::unique_lock & lock, const std::function & is_cancelled); - BlockIO tryToExecuteQuery(const String & query_to_execute, const String & comment); + void dumpDataForTables(mysqlxx::Pool::Entry & connection, MaterializeMetadata & master_info, const std::function & is_cancelled); - String getCreateQuery(const mysqlxx::Pool::Entry & connection, const String & table_name); - - void dumpDataForTables(mysqlxx::Pool::Entry & connection, const std::vector & tables_name, const std::function & is_cancelled); - - mutable std::mutex sync_mutex; + std::mutex sync_mutex; std::atomic sync_quit{false}; std::condition_variable sync_cond; - ThreadFromGlobalPool thread{&DatabaseMaterializeMySQL::synchronized, this}; + ThreadPool background_thread_pool{1}; }; } diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQLWrap.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQLWrap.cpp new file mode 100644 index 00000000000..6b628ddf26d --- /dev/null +++ b/src/Databases/MySQL/DatabaseMaterializeMySQLWrap.cpp @@ -0,0 +1,188 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +DatabaseMaterializeMySQLWrap::DatabaseMaterializeMySQLWrap(const DatabasePtr & nested_database_, const ASTPtr & database_engine_define_, const String & log_name) + : IDatabase(nested_database_->getDatabaseName()), nested_database(nested_database_), database_engine_define(database_engine_define_), log(&Logger::get(log_name)) +{ +} + +void DatabaseMaterializeMySQLWrap::setException(const std::exception_ptr & exception_) +{ + std::unique_lock lock(mutex); + exception = exception_; +} + +DatabasePtr DatabaseMaterializeMySQLWrap::getNestedDatabase() const +{ + std::unique_lock lock(mutex); + + if (exception) + std::rethrow_exception(exception); + + return nested_database; +} + +ASTPtr DatabaseMaterializeMySQLWrap::getCreateDatabaseQuery() const +{ + const auto & create_query = std::make_shared(); + create_query->database = database_name; + create_query->set(create_query->storage, database_engine_define); + return create_query; +} +void DatabaseMaterializeMySQLWrap::loadStoredObjects(Context & context, bool has_force_restore_data_flag) +{ + try + { + LOG_DEBUG(log, "Loading MySQL nested database stored objects."); + getNestedDatabase()->loadStoredObjects(context, has_force_restore_data_flag); + LOG_DEBUG(log, "Loaded MySQL nested database stored objects."); + } + catch (...) + { + tryLogCurrentException(log, "Cannot load MySQL nested database stored objects."); + throw; + } +} + +void DatabaseMaterializeMySQLWrap::shutdown() +{ + getNestedDatabase()->shutdown(); +} + +bool DatabaseMaterializeMySQLWrap::empty() const +{ + return getNestedDatabase()->empty(); +} + +String DatabaseMaterializeMySQLWrap::getDataPath() const +{ + return getNestedDatabase()->getDataPath(); +} + +String DatabaseMaterializeMySQLWrap::getMetadataPath() const +{ + return getNestedDatabase()->getMetadataPath(); +} + +String DatabaseMaterializeMySQLWrap::getTableDataPath(const String & table_name) const +{ + return getNestedDatabase()->getTableDataPath(table_name); +} + +String DatabaseMaterializeMySQLWrap::getTableDataPath(const ASTCreateQuery & query) const +{ + return getNestedDatabase()->getTableDataPath(query); +} + +String DatabaseMaterializeMySQLWrap::getObjectMetadataPath(const String & table_name) const +{ + return getNestedDatabase()->getObjectMetadataPath(table_name); +} + +UUID DatabaseMaterializeMySQLWrap::tryGetTableUUID(const String & table_name) const +{ + return getNestedDatabase()->tryGetTableUUID(table_name); +} + +time_t DatabaseMaterializeMySQLWrap::getObjectMetadataModificationTime(const String & name) const +{ + return getNestedDatabase()->getObjectMetadataModificationTime(name); +} + +void DatabaseMaterializeMySQLWrap::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + throw Exception("MySQL database in locality_data mode does not support create table.", ErrorCodes::NOT_IMPLEMENTED); + + getNestedDatabase()->createTable(context, name, table, query); +} + +void DatabaseMaterializeMySQLWrap::dropTable(const Context & context, const String & name, bool no_delay) +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + throw Exception("MySQL database in locality_data mode does not support drop table.", ErrorCodes::NOT_IMPLEMENTED); + + getNestedDatabase()->dropTable(context, name, no_delay); +} + +void DatabaseMaterializeMySQLWrap::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + throw Exception("MySQL database in locality_data mode does not support attach table.", ErrorCodes::NOT_IMPLEMENTED); + + getNestedDatabase()->attachTable(name, table, relative_table_path); +} + +StoragePtr DatabaseMaterializeMySQLWrap::detachTable(const String & name) +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + throw Exception("MySQL database in locality_data mode does not support detach table.", ErrorCodes::NOT_IMPLEMENTED); + + return getNestedDatabase()->detachTable(name); +} + +void DatabaseMaterializeMySQLWrap::renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange) +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + throw Exception("MySQL database in locality_data mode does not support rename table.", ErrorCodes::NOT_IMPLEMENTED); + + getNestedDatabase()->renameTable(context, name, to_database, to_name, exchange); +} + +void DatabaseMaterializeMySQLWrap::alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + throw Exception("MySQL database in locality_data mode does not support alter table.", ErrorCodes::NOT_IMPLEMENTED); + + getNestedDatabase()->alterTable(context, table_id, metadata); +} + +bool DatabaseMaterializeMySQLWrap::shouldBeEmptyOnDetach() const +{ + return getNestedDatabase()->shouldBeEmptyOnDetach(); +} + +void DatabaseMaterializeMySQLWrap::drop(const Context & context) +{ + getNestedDatabase()->drop(context); +} + +bool DatabaseMaterializeMySQLWrap::isTableExist(const String & name) const +{ + return getNestedDatabase()->isTableExist(name); +} + +StoragePtr DatabaseMaterializeMySQLWrap::tryGetTable(const String & name) const +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + return std::make_shared(getNestedDatabase()->tryGetTable(name)); + + return getNestedDatabase()->tryGetTable(name); +} + +DatabaseTablesIteratorPtr DatabaseMaterializeMySQLWrap::getTablesIterator(const FilterByNameFunction & filter_by_table_name) +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + { + DatabaseTablesIteratorPtr iterator = getNestedDatabase()->getTablesIterator(filter_by_table_name); + return std::make_unique(std::move(iterator)); + } + + return getNestedDatabase()->getTablesIterator(filter_by_table_name); +} + +} diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQLWrap.h b/src/Databases/MySQL/DatabaseMaterializeMySQLWrap.h new file mode 100644 index 00000000000..0f16661c66f --- /dev/null +++ b/src/Databases/MySQL/DatabaseMaterializeMySQLWrap.h @@ -0,0 +1,70 @@ +#pragma once + +#include + +namespace DB +{ + +class DatabaseMaterializeMySQLWrap : public IDatabase +{ +public: + ASTPtr getCreateDatabaseQuery() const override; + + void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override; + + DatabaseMaterializeMySQLWrap(const DatabasePtr & nested_database_, const ASTPtr & database_engine_define_, const String & log_name); +protected: + DatabasePtr nested_database; + ASTPtr database_engine_define; + Poco::Logger * log; + + mutable std::mutex mutex; + std::exception_ptr exception; + + DatabasePtr getNestedDatabase() const; + + void setException(const std::exception_ptr & exception); + +public: + void shutdown() override; + + bool empty() const override; + + String getDataPath() const override; + + String getTableDataPath(const String & table_name) const override; + + String getTableDataPath(const ASTCreateQuery & query) const override; + + UUID tryGetTableUUID(const String & table_name) const override; + + void createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) override; + + void dropTable(const Context & context, const String & name, bool no_delay) override; + + void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) override; + + StoragePtr detachTable(const String & name) override; + + void renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange) override; + + void alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; + + time_t getObjectMetadataModificationTime(const String & name) const override; + + String getMetadataPath() const override; + + String getObjectMetadataPath(const String & table_name) const override; + + bool shouldBeEmptyOnDetach() const override; + + void drop(const Context & context) override; + + bool isTableExist(const String & name) const override; + + StoragePtr tryGetTable(const String & name) const override; + + DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; +}; + +} diff --git a/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h b/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h new file mode 100644 index 00000000000..805ae3e4c6e --- /dev/null +++ b/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h @@ -0,0 +1,34 @@ +#pragma once + +#include + +namespace DB +{ + +class DatabaseMaterializeTablesIterator final : public IDatabaseTablesIterator +{ +public: + virtual void next() { nested_iterator->next(); } + + virtual bool isValid() const { return nested_iterator->isValid(); } + + virtual const String & name() const { return nested_iterator->name(); } + + virtual const StoragePtr & table() const + { + StoragePtr storage = std::make_shared(nested_iterator->table()); + return tables.emplace_back(storage); + } + + virtual UUID uuid() const { return nested_iterator->uuid(); } + + DatabaseMaterializeTablesIterator(DatabaseTablesIteratorPtr nested_iterator_) : nested_iterator(std::move(nested_iterator_)) + {} + +private: + mutable std::vector tables; + DatabaseTablesIteratorPtr nested_iterator; + +}; + +} diff --git a/src/Databases/MySQL/EventConsumer.cpp b/src/Databases/MySQL/EventConsumer.cpp new file mode 100644 index 00000000000..9671464fd2c --- /dev/null +++ b/src/Databases/MySQL/EventConsumer.cpp @@ -0,0 +1,261 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +using namespace MySQLReplication; + +EventConsumer::~EventConsumer() +{ + if (!quit && !background_exception) + { + { + quit = true; + std::lock_guard lock(mutex); + } + + cond.notify_one(); + background_thread_pool.wait(); + } +} + +EventConsumer::EventConsumer( + const String & database_, const Context & context_, MaterializeMetadata & metadata_, MaterializeModeSettings & settings_) + : metadata(metadata_), context(context_), settings(settings_), database(database_), prev_version(metadata.version) +{ + background_thread_pool.scheduleOrThrowOnError([&]() + { + ThreadStatus thread_status; + setThreadName("MySQLDBSync"); + std::unique_lock lock(mutex); + const auto quit_requested = [this] { return quit.load(std::memory_order_relaxed); }; + + while (!quit_requested() && !background_exception) + { + if (!buffers.empty() && total_bytes_in_buffers) + flushBuffers(); + + cond.wait_for(lock, std::chrono::milliseconds(settings.max_flush_data_time), quit_requested); + } + }); +} + +void EventConsumer::onWriteData(const String & table_name, const std::vector & rows_data) +{ + BufferPtr buffer = getTableBuffer(table_name); + + size_t prev_bytes = buffer->data.bytes(); + for (size_t column = 0; column < buffer->data.columns() - 2; ++column) + { + MutableColumnPtr col_to = (*std::move(buffer->data.getByPosition(column).column)).mutate(); + + for (size_t index = 0; index < rows_data.size(); ++index) + col_to->insert(DB::get(rows_data[index])[column]); + } + + fillSignColumnsAndMayFlush(buffer->data, 1, ++metadata.version, rows_data.size(), prev_bytes); +} + +static inline bool differenceSortingKeys(const Tuple & row_old_data, const Tuple & row_new_data, const std::vector sorting_columns_index) +{ + for (const auto & sorting_column_index : sorting_columns_index) + if (row_old_data[sorting_column_index] != row_new_data[sorting_column_index]) + return true; + + return false; +} + +void EventConsumer::onUpdateData(const String & table_name, const std::vector & rows_data) +{ + if (rows_data.size() % 2 != 0) + throw Exception("LOGICAL ERROR: ", ErrorCodes::LOGICAL_ERROR); + + BufferPtr buffer = getTableBuffer(table_name); + + size_t prev_bytes = buffer->data.bytes(); + std::vector difference_sorting_keys_mark(rows_data.size() / 2); + + for (size_t index = 0; index < rows_data.size(); index += 2) + difference_sorting_keys_mark.emplace_back(differenceSortingKeys( + DB::get(rows_data[index]), DB::get(rows_data[index + 1]), buffer->sorting_columns_index)); + + for (size_t column = 0; column < buffer->data.columns() - 2; ++column) + { + MutableColumnPtr col_to = (*std::move(buffer->data.getByPosition(column).column)).mutate(); + + for (size_t index = 0; index < rows_data.size(); index += 2) + { + if (likely(!difference_sorting_keys_mark[index / 2])) + col_to->insert(DB::get(rows_data[index + 1])[column]); + else + { + /// If the sorting keys is modified, we should cancel the old data, but this should not happen frequently + col_to->insert(DB::get(rows_data[index])[column]); + col_to->insert(DB::get(rows_data[index + 1])[column]); + } + } + } + + MutableColumnPtr sign_mutable_column = (*std::move(buffer->data.getByPosition(buffer->data.columns() - 2)).column).mutate(); + MutableColumnPtr version_mutable_column = (*std::move(buffer->data.getByPosition(buffer->data.columns() - 1)).column).mutate(); + + ColumnInt8::Container & sign_column_data = assert_cast(*sign_mutable_column).getData(); + ColumnUInt64::Container & version_column_data = assert_cast(*version_mutable_column).getData(); + + UInt64 new_version = ++metadata.version; + for (size_t index = 0; index < rows_data.size(); index += 2) + { + if (likely(!difference_sorting_keys_mark[index / 2])) + { + sign_column_data.emplace_back(1); + version_column_data.emplace_back(new_version); + } + else + { + /// If the sorting keys is modified, we should cancel the old data, but this should not happen frequently + sign_column_data.emplace_back(-1); + sign_column_data.emplace_back(1); + version_column_data.emplace_back(new_version); + version_column_data.emplace_back(new_version); + } + } + + total_bytes_in_buffers += (buffer->data.bytes() - prev_bytes); + if (buffer->data.rows() >= settings.max_rows_in_buffer || total_bytes_in_buffers >= settings.max_bytes_in_buffers) + flushBuffers(); +} + +void EventConsumer::onDeleteData(const String & table_name, const std::vector & rows_data) +{ + BufferPtr buffer = getTableBuffer(table_name); + + size_t prev_bytes = buffer->data.bytes(); + for (size_t column = 0; column < buffer->data.columns() - 2; ++column) + { + MutableColumnPtr col_to = (*std::move(buffer->data.getByPosition(column).column)).mutate(); + + for (size_t index = 0; index < rows_data.size(); ++index) + col_to->insert(DB::get(rows_data[index])[column]); + } + + fillSignColumnsAndMayFlush(buffer->data, -1, ++metadata.version, rows_data.size(), prev_bytes); +} + +EventConsumer::BufferPtr EventConsumer::getTableBuffer(const String & table_name) +{ + if (buffers.find(table_name) == buffers.end()) + { + StoragePtr storage = DatabaseCatalog::instance().getDatabase(database)->tryGetTable(table_name); + + buffers[table_name] = std::make_shared(); + buffers[table_name]->data = storage->getSampleBlockNonMaterialized(); + if (StorageMergeTree * table_merge_tree = dynamic_cast(storage.get())) + { + Names required_for_sorting_key = table_merge_tree->getColumnsRequiredForSortingKey(); + + for (const auto & required_name_for_sorting_key : required_for_sorting_key) + buffers[table_name]->sorting_columns_index.emplace_back( + buffers[table_name]->data.getPositionByName(required_name_for_sorting_key)); + } + } + + return buffers[table_name]; +} + +void EventConsumer::onEvent(const BinlogEventPtr & receive_event, const MySQLReplication::Position & position) +{ + std::unique_lock lock(mutex); + + if (background_exception) + background_thread_pool.wait(); + + last_position = position; + if (receive_event->type() == MYSQL_WRITE_ROWS_EVENT) + { + WriteRowsEvent & write_rows_event = static_cast(*receive_event); + write_rows_event.dump(); + onWriteData(write_rows_event.table, write_rows_event.rows); + } + else if (receive_event->type() == MYSQL_UPDATE_ROWS_EVENT) + { + UpdateRowsEvent & update_rows_event = static_cast(*receive_event); + update_rows_event.dump(); + onUpdateData(update_rows_event.table, update_rows_event.rows); + } + else if (receive_event->type() == MYSQL_DELETE_ROWS_EVENT) + { + DeleteRowsEvent & delete_rows_event = static_cast(*receive_event); + delete_rows_event.dump(); + onDeleteData(delete_rows_event.table, delete_rows_event.rows); + } + else if (receive_event->type() == MYSQL_QUERY_EVENT) + { + /// TODO: 识别, 查看是否支持的DDL, 支持的话立即刷新当前的数据, 然后执行DDL. +// flush_function(); + /// TODO: 直接使用Interpreter执行即可 + } +} + +void EventConsumer::fillSignColumnsAndMayFlush(Block & data, Int8 sign_value, UInt64 version_value, size_t fill_size, size_t prev_bytes) +{ + MutableColumnPtr sign_mutable_column = (*std::move(data.getByPosition(data.columns() - 2)).column).mutate(); + MutableColumnPtr version_mutable_column = (*std::move(data.getByPosition(data.columns() - 1)).column).mutate(); + + ColumnInt8::Container & sign_column_data = assert_cast(*sign_mutable_column).getData(); + ColumnUInt64::Container & version_column_data = assert_cast(*version_mutable_column).getData(); + + for (size_t index = 0; index < fill_size; ++index) + { + sign_column_data.emplace_back(sign_value); + version_column_data.emplace_back(version_value); + } + + total_bytes_in_buffers += (data.bytes() - prev_bytes); + if (data.rows() >= settings.max_rows_in_buffer || total_bytes_in_buffers >= settings.max_bytes_in_buffers) + flushBuffers(); +} + +void EventConsumer::flushBuffers() +{ + /// TODO: 事务保证 + try + { + for (auto & table_name_and_buffer : buffers) + { + const String & table_name = table_name_and_buffer.first; + BufferPtr & buffer = table_name_and_buffer.second; + + Context query_context = context; + query_context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + query_context.setCurrentQueryId(""); // generate random query_id + String with_database_table_name = backQuoteIfNeed(database) + "." + backQuoteIfNeed(table_name); + BlockIO res = executeQuery("INSERT INTO " + with_database_table_name + " VALUES", query_context, true); + + OneBlockInputStream input(buffer->data); + copyData(input, *res.out); + } + + buffers.clear(); + total_bytes_in_buffers = 0; + prev_version = metadata.version; + } + catch(...) + { + buffers.clear(); + total_bytes_in_buffers = 0; + metadata.version = prev_version; + background_exception = true; + throw; + } +} + +} diff --git a/src/Databases/MySQL/EventConsumer.h b/src/Databases/MySQL/EventConsumer.h new file mode 100644 index 00000000000..f21265efe9d --- /dev/null +++ b/src/Databases/MySQL/EventConsumer.h @@ -0,0 +1,63 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include "MaterializeMetadata.h" + +namespace DB +{ + +class EventConsumer : private boost::noncopyable +{ +public: + ~EventConsumer(); + + void onEvent(const MySQLReplication::BinlogEventPtr & event, const MySQLReplication::Position & position); + + EventConsumer(const String & database_, const Context & context, MaterializeMetadata & metadata_, MaterializeModeSettings & settings_); +private: + MaterializeMetadata & metadata; + + const Context & context; + const MaterializeModeSettings & settings; + + String database; + size_t prev_version; + size_t total_bytes_in_buffers = 0; + MySQLReplication::Position last_position; + + struct Buffer + { + Block data; + std::vector sorting_columns_index; + }; + + using BufferPtr = std::shared_ptr; + std::unordered_map buffers; + + void flushBuffers(); + + BufferPtr getTableBuffer(const String & table_name); + + void onWriteData(const std::string & table_name, const std::vector & rows_data); + + void onUpdateData(const std::string & table_name, const std::vector & rows_data); + + void onDeleteData(const std::string & table_name, const std::vector & rows_data); + + void fillSignColumnsAndMayFlush(Block & data, Int8 sign_value, UInt64 version_value, size_t fill_size, size_t prev_bytes); + + mutable std::mutex mutex; + std::condition_variable cond; + std::atomic_bool quit = false; + std::atomic_bool background_exception = false; + ThreadPool background_thread_pool{1}; +}; + +} diff --git a/src/Databases/MySQL/MasterStatusInfo.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp similarity index 52% rename from src/Databases/MySQL/MasterStatusInfo.cpp rename to src/Databases/MySQL/MaterializeMetadata.cpp index 687a028ebdc..768ed4fc57b 100644 --- a/src/Databases/MySQL/MasterStatusInfo.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -1,17 +1,44 @@ #include #include #include -#include +#include #include -#include -#include -#include #include #include +#include +#include +#include +#include namespace DB { +static std::unordered_map fetchTablesCreateQuery( + const mysqlxx::PoolWithFailover::Entry & connection, const String & database_name, const std::vector & fetch_tables) +{ + std::unordered_map tables_create_query; + for (size_t index = 0; index < fetch_tables.size(); ++index) + { + Block show_create_table_header{ + {std::make_shared(), "Table"}, + {std::make_shared(), "Create Table"}, + }; + + MySQLBlockInputStream show_create_table( + connection, "SHOW CREATE TABLE " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(fetch_tables[index]), + show_create_table_header, DEFAULT_BLOCK_SIZE); + + Block create_query_block = show_create_table.read(); + if (!create_query_block || create_query_block.rows() != 1) + throw Exception("LOGICAL ERROR mysql show create return more rows.", ErrorCodes::LOGICAL_ERROR); + + tables_create_query[fetch_tables[index]] = create_query_block.getByName("Create Table").column->getDataAt(0).toString(); + } + + return tables_create_query; +} + + static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entry & connection, const std::string & database) { Block header{{std::make_shared(), "table_name"}}; @@ -29,7 +56,7 @@ static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entr return tables_in_db; } -void MasterStatusInfo::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & connection) +void MaterializeMetadata::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & connection) { Block header{ {std::make_shared(), "File"}, @@ -45,6 +72,7 @@ void MasterStatusInfo::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & conn if (!master_status || master_status.rows() != 1) throw Exception("Unable to get master status from MySQL.", ErrorCodes::LOGICAL_ERROR); + version = 0; binlog_file = (*master_status.getByPosition(0).column)[0].safeGet(); binlog_position = (*master_status.getByPosition(1).column)[0].safeGet(); binlog_do_db = (*master_status.getByPosition(2).column)[0].safeGet(); @@ -52,7 +80,7 @@ void MasterStatusInfo::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & conn executed_gtid_set = (*master_status.getByPosition(4).column)[0].safeGet(); } -bool MasterStatusInfo::checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection) +bool MaterializeMetadata::checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection) { Block header{ {std::make_shared(), "Log_name"}, @@ -73,53 +101,65 @@ bool MasterStatusInfo::checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & } return false; } -void MasterStatusInfo::finishDump() -{ - WriteBufferFromFile out(persistent_path); - out << "Version:\t1\n" - << "Binlog File:\t" << binlog_file << "\nBinlog Position:\t" << binlog_position << "\nBinlog Do DB:\t" << binlog_do_db - << "\nBinlog Ignore DB:\t" << binlog_ignore_db << "\nExecuted GTID SET:\t" << executed_gtid_set; - out.next(); - out.sync(); +void commitMetadata(const std::function & function, const String & persistent_tmp_path, const String & persistent_path) +{ + try + { + function(); + + Poco::File(persistent_tmp_path).renameTo(persistent_path); + } + catch (...) + { + Poco::File(persistent_tmp_path).remove(); + throw; + } } -void MasterStatusInfo::transaction(const MySQLReplication::Position & position, const std::function & fun) +void MaterializeMetadata::transaction(const MySQLReplication::Position & position, const std::function & fun) { binlog_file = position.binlog_name; binlog_position = position.binlog_pos; + String persistent_tmp_path = persistent_path + ".tmp"; + { - Poco::File temp_file(persistent_path + ".temp"); - if (temp_file.exists()) - temp_file.remove(); + WriteBufferFromFile out(persistent_tmp_path, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_TRUNC | O_CREAT | O_EXCL); + + /// TSV format metadata file. + writeString("Version:\t1\n", out); + writeString("Binlog File:\t" + binlog_file + "\n", out); + writeString("Executed GTID:\t" + executed_gtid_set + "\n", out); + writeString("Binlog Position:\t" + toString(binlog_position) + "\n", out); + writeString("Data Version:\t" + toString(version) + "\n", out); + + out.next(); + out.sync(); + out.close(); } - WriteBufferFromFile out(persistent_path + ".temp"); - out << "Version:\t1\n" - << "Binlog File:\t" << binlog_file << "\nBinlog Position:\t" << binlog_position << "\nBinlog Do DB:\t" << binlog_do_db - << "\nBinlog Ignore DB:\t" << binlog_ignore_db << "\nExecuted GTID SET:\t" << executed_gtid_set; - out.next(); - out.sync(); - - fun(); - Poco::File(persistent_path + ".temp").renameTo(persistent_path); + commitMetadata(fun, persistent_tmp_path, persistent_path); } -MasterStatusInfo::MasterStatusInfo(mysqlxx::PoolWithFailover::Entry & connection, const String & path_, const String & database) +MaterializeMetadata::MaterializeMetadata(mysqlxx::PoolWithFailover::Entry & connection, const String & path_, const String & database) : persistent_path(path_) { if (Poco::File(persistent_path).exists()) { - ReadBufferFromFile in(persistent_path); - in >> "Version:\t1\n" >> "Binlog File:\t" >> binlog_file >> "\nBinlog Position:\t" >> binlog_position >> "\nBinlog Do DB:\t" - >> binlog_do_db >> "\nBinlog Ignore DB:\t" >> binlog_ignore_db >> "\nExecuted GTID SET:\t" >> executed_gtid_set; + ReadBufferFromFile in(persistent_path, DBMS_DEFAULT_BUFFER_SIZE); + assertString("Version:\t1\n", in); + assertString("Binlog File:\t", in); + readString(binlog_file, in); + assertString("Executed GTID:\t", in); + readString(executed_gtid_set, in); + assertString("Binlog Position:\t", in); + readIntText(binlog_position, in); + assertString("Data Version:\t", in); + readIntText(version, in); if (checkBinlogFileExists(connection)) - { - std::cout << "Load From File \n"; return; - } } bool locked_tables = false; @@ -134,9 +174,8 @@ MasterStatusInfo::MasterStatusInfo(mysqlxx::PoolWithFailover::Entry & connection connection->query("SET SESSION TRANSACTION ISOLATION LEVEL REPEATABLE READ;").execute(); connection->query("START TRANSACTION /*!40100 WITH CONSISTENT SNAPSHOT */;").execute(); - need_dumping_tables = fetchTablesInDB(connection, database); + need_dumping_tables = fetchTablesCreateQuery(connection, database, fetchTablesInDB(connection, database)); connection->query("UNLOCK TABLES;").execute(); - /// TODO: 拉取建表语句, 解析并构建出表结构(列列表, 主键, 唯一索引, 分区键) } catch (...) { diff --git a/src/Databases/MySQL/MasterStatusInfo.h b/src/Databases/MySQL/MaterializeMetadata.h similarity index 72% rename from src/Databases/MySQL/MasterStatusInfo.h rename to src/Databases/MySQL/MaterializeMetadata.h index d18caf4b5fe..ad2f77a2f42 100644 --- a/src/Databases/MySQL/MasterStatusInfo.h +++ b/src/Databases/MySQL/MaterializeMetadata.h @@ -8,7 +8,7 @@ namespace DB { -struct MasterStatusInfo +struct MaterializeMetadata { const String persistent_path; @@ -18,9 +18,8 @@ struct MasterStatusInfo String binlog_ignore_db; String executed_gtid_set; - std::vector need_dumping_tables; - - void finishDump(); + size_t version = 0; + std::unordered_map need_dumping_tables; void fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & connection); @@ -28,9 +27,7 @@ struct MasterStatusInfo void transaction(const MySQLReplication::Position & position, const std::function & fun); - MasterStatusInfo(mysqlxx::PoolWithFailover::Entry & connection, const String & path, const String & database); - - + MaterializeMetadata(mysqlxx::PoolWithFailover::Entry & connection, const String & path, const String & database); }; } diff --git a/src/Databases/MySQL/MaterializeModeSettings.cpp b/src/Databases/MySQL/MaterializeModeSettings.cpp new file mode 100644 index 00000000000..d8a25fa3bb9 --- /dev/null +++ b/src/Databases/MySQL/MaterializeModeSettings.cpp @@ -0,0 +1,42 @@ +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_COLLECTION(MaterializeModeSettings, LIST_OF_MATERIALIZE_MODE_SETTINGS) + +void MaterializeModeSettings::loadFromQuery(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + throw Exception(e.message() + " for storage " + storage_def.engine->name, ErrorCodes::BAD_ARGUMENTS); + else + e.rethrow(); + } + } + else + { + auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; + storage_def.set(storage_def.settings, settings_ast); + } +} + +} diff --git a/src/Databases/MySQL/MaterializeModeSettings.h b/src/Databases/MySQL/MaterializeModeSettings.h new file mode 100644 index 00000000000..35c65253c87 --- /dev/null +++ b/src/Databases/MySQL/MaterializeModeSettings.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ASTStorage; + +/** Settings for the MySQL Database engine(materialize mode). + * Could be loaded from a CREATE DATABASE query (SETTINGS clause). + */ +struct MaterializeModeSettings : public SettingsCollection +{ +#define LIST_OF_MATERIALIZE_MODE_SETTINGS(M) \ + M(SettingBool, locality_data, false, "", 0) \ + M(SettingUInt64, max_rows_in_buffer, DEFAULT_BLOCK_SIZE, "", 0) \ + M(SettingUInt64, max_bytes_in_buffers, DBMS_DEFAULT_BUFFER_SIZE, "", 0) \ + M(SettingUInt64, max_flush_data_time, 1000, "", 0) \ + M(SettingUInt64, max_wait_time_when_mysql_unavailable, 1000, "", 0) \ + + DECLARE_SETTINGS_COLLECTION(LIST_OF_MATERIALIZE_MODE_SETTINGS) + + void loadFromQuery(ASTStorage & storage_def); +}; + +} diff --git a/src/Databases/MySQL/queryConvert.cpp b/src/Databases/MySQL/queryConvert.cpp index decbf8f3cb6..0414c654bd6 100644 --- a/src/Databases/MySQL/queryConvert.cpp +++ b/src/Databases/MySQL/queryConvert.cpp @@ -137,13 +137,16 @@ String getUniqueColumnName(NamesAndTypesList columns_name_and_type, const String String toCreateQuery(const MySQLTableStruct & table_struct, const Context & context) { /// TODO: settings - String create_version = getUniqueColumnName(table_struct.columns_name_and_type, "_create_version"); - String delete_version = getUniqueColumnName(table_struct.columns_name_and_type, "_delete_version"); + if (table_struct.primary_keys.empty()) + throw Exception("", ErrorCodes::NOT_IMPLEMENTED); + WriteBufferFromOwnString out; + String sign = getUniqueColumnName(table_struct.columns_name_and_type, "__sign"); + String version = getUniqueColumnName(table_struct.columns_name_and_type, "__version"); out << "CREATE TABLE " << (table_struct.if_not_exists ? "IF NOT EXISTS" : "") - << backQuoteIfNeed(table_struct.database_name) + "." << backQuoteIfNeed(table_struct.table_name) << "(" - << queryToString(InterpreterCreateQuery::formatColumns(table_struct.columns_name_and_type)) - << ", " << create_version << " UInt64, " << delete_version << " UInt64" << ") ENGINE = MergeTree()" + << backQuoteIfNeed(table_struct.database_name) + "." << backQuoteIfNeed(table_struct.table_name) + << "(" << queryToString(InterpreterCreateQuery::formatColumns(table_struct.columns_name_and_type)) + << ", " << sign << " Int8, " << version << " UInt64" << ") ENGINE = ReplacingMergeTree(" + version + ")" << " PARTITION BY " << queryToString(getFormattedPartitionByExpression(table_struct, context, 1000, 50000)) << " ORDER BY " << queryToString(getFormattedOrderByExpression(table_struct)); return out.str(); diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index 30d681cb126..8bfd426b03f 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -28,7 +28,11 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p if (!s_eq.ignore(pos, expected)) return false; - if (!value_p.parse(pos, value, expected)) + if (ParserKeyword("TRUE").ignore(pos, expected)) + value = std::make_shared(Field(UInt64(1))); + else if (ParserKeyword("FALSE").ignore(pos, expected)) + value = std::make_shared(Field(UInt64(0))); + else if (!value_p.parse(pos, value, expected)) return false; tryGetIdentifierNameInto(name, change.name); diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp new file mode 100644 index 00000000000..6bcdc9bd3b8 --- /dev/null +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -0,0 +1,56 @@ +#include + +#include +#include + +#include + +namespace DB +{ + +StorageMaterializeMySQL::StorageMaterializeMySQL(const StoragePtr & nested_storage_) + : IStorage(nested_storage_->getStorageID()), nested_storage(nested_storage_) +{ + ColumnsDescription columns_desc; + const ColumnsDescription & nested_columns_desc = nested_storage->getColumns(); + + size_t index = 0; + auto iterator = nested_columns_desc.begin(); + for (; index < nested_columns_desc.size() - 2; ++index, ++iterator) + columns_desc.add(*iterator); + + setColumns(columns_desc); +} + +Pipes StorageMaterializeMySQL::read( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned int num_streams) +{ + if (ASTSelectQuery * select_query = query_info.query->as()) + { + auto & tables_in_select_query = select_query->tables()->as(); + + if (!tables_in_select_query.children.empty()) + { + auto & tables_element = tables_in_select_query.children[0]->as(); + + if (tables_element.table_expression) + tables_element.table_expression->as().final = true; + } + } + + Names require_columns_name = column_names; + Block header = nested_storage->getSampleBlockNonMaterialized(); + ColumnWithTypeAndName & sign_column = header.getByPosition(header.columns() - 2); + + if (require_columns_name.end() == std::find(require_columns_name.begin(), require_columns_name.end(), sign_column.name)) + require_columns_name.emplace_back(sign_column.name); + + return nested_storage->read(require_columns_name, query_info, context, processed_stage, max_block_size, num_streams); +} + +} diff --git a/src/Storages/StorageMaterializeMySQL.h b/src/Storages/StorageMaterializeMySQL.h new file mode 100644 index 00000000000..d88e72ed737 --- /dev/null +++ b/src/Storages/StorageMaterializeMySQL.h @@ -0,0 +1,28 @@ +#pragma once + +#include + +namespace DB +{ + +class StorageMaterializeMySQL final : public ext::shared_ptr_helper, public IStorage +{ + friend struct ext::shared_ptr_helper; +public: + String getName() const override { return "MySQL"; } + + bool supportsFinal() const override { return nested_storage->supportsFinal(); } + bool supportsSampling() const override { return nested_storage->supportsSampling(); } + + + StorageMaterializeMySQL(const StoragePtr & nested_storage_); + + Pipes read( + const Names & column_names, const SelectQueryInfo & query_info, + const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; + +private: + StoragePtr nested_storage; +}; + +} From 59465b58c235a9ed62b82102600c37eb1d50cae7 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 19 Jun 2020 19:59:16 +0800 Subject: [PATCH 068/374] ISSUES-4006 fix cmake --- src/Core/tests/CMakeLists.txt | 8 -------- src/DataTypes/DataTypeEnum.h | 3 +-- 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/src/Core/tests/CMakeLists.txt b/src/Core/tests/CMakeLists.txt index b24326d94b1..d609e49f247 100644 --- a/src/Core/tests/CMakeLists.txt +++ b/src/Core/tests/CMakeLists.txt @@ -16,16 +16,8 @@ if (ENABLE_FUZZING) target_link_libraries (names_and_types_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) endif () -add_executable (mysql_client mysql_client.cpp) -target_link_libraries (mysql_client PRIVATE dbms) - add_executable (mysql_protocol mysql_protocol.cpp) target_link_libraries (mysql_protocol PRIVATE dbms) if(USE_SSL) target_include_directories (mysql_protocol SYSTEM PRIVATE ${OPENSSL_INCLUDE_DIR}) endif() - -if (ENABLE_FUZZING) - add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp) - target_link_libraries (names_and_types_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) -endif () diff --git a/src/DataTypes/DataTypeEnum.h b/src/DataTypes/DataTypeEnum.h index 4756a218778..80b41692cdd 100644 --- a/src/DataTypes/DataTypeEnum.h +++ b/src/DataTypes/DataTypeEnum.h @@ -43,8 +43,7 @@ public: using ColumnType = ColumnVector; using Value = std::pair; using Values = std::vector; - using NameToValueMap = HashMap; + using NameToValueMap = HashMap; using ValueToNameMap = std::unordered_map; static constexpr bool is_parametric = true; From 0f75b2d0f0e8c0e7f8e5c503ac558d62f88115eb Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 19 Jun 2020 20:50:52 +0800 Subject: [PATCH 069/374] ISSUES-4006 fix buidl failure --- src/Databases/MySQL/EventConsumer.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Databases/MySQL/EventConsumer.cpp b/src/Databases/MySQL/EventConsumer.cpp index 9671464fd2c..2054f5810ef 100644 --- a/src/Databases/MySQL/EventConsumer.cpp +++ b/src/Databases/MySQL/EventConsumer.cpp @@ -56,7 +56,7 @@ void EventConsumer::onWriteData(const String & table_name, const std::vectordata.bytes(); for (size_t column = 0; column < buffer->data.columns() - 2; ++column) { - MutableColumnPtr col_to = (*std::move(buffer->data.getByPosition(column).column)).mutate(); + MutableColumnPtr col_to = IColumn::mutate(std::move(buffer->data.getByPosition(column).column)); for (size_t index = 0; index < rows_data.size(); ++index) col_to->insert(DB::get(rows_data[index])[column]); @@ -90,7 +90,7 @@ void EventConsumer::onUpdateData(const String & table_name, const std::vectordata.columns() - 2; ++column) { - MutableColumnPtr col_to = (*std::move(buffer->data.getByPosition(column).column)).mutate(); + MutableColumnPtr col_to = IColumn::mutate(std::move(buffer->data.getByPosition(column).column)); for (size_t index = 0; index < rows_data.size(); index += 2) { @@ -105,8 +105,8 @@ void EventConsumer::onUpdateData(const String & table_name, const std::vectordata.getByPosition(buffer->data.columns() - 2)).column).mutate(); - MutableColumnPtr version_mutable_column = (*std::move(buffer->data.getByPosition(buffer->data.columns() - 1)).column).mutate(); + MutableColumnPtr sign_mutable_column = IColumn::mutate(std::move(buffer->data.getByPosition(buffer->data.columns() - 2).column)); + MutableColumnPtr version_mutable_column = IColumn::mutate(std::move(buffer->data.getByPosition(buffer->data.columns() - 1).column)); ColumnInt8::Container & sign_column_data = assert_cast(*sign_mutable_column).getData(); ColumnUInt64::Container & version_column_data = assert_cast(*version_mutable_column).getData(); @@ -141,7 +141,7 @@ void EventConsumer::onDeleteData(const String & table_name, const std::vectordata.bytes(); for (size_t column = 0; column < buffer->data.columns() - 2; ++column) { - MutableColumnPtr col_to = (*std::move(buffer->data.getByPosition(column).column)).mutate(); + MutableColumnPtr col_to = IColumn::mutate(std::move(buffer->data.getByPosition(column).column)); for (size_t index = 0; index < rows_data.size(); ++index) col_to->insert(DB::get(rows_data[index])[column]); @@ -154,7 +154,7 @@ EventConsumer::BufferPtr EventConsumer::getTableBuffer(const String & table_name { if (buffers.find(table_name) == buffers.end()) { - StoragePtr storage = DatabaseCatalog::instance().getDatabase(database)->tryGetTable(table_name); + StoragePtr storage = DatabaseCatalog::instance().getDatabase(database)->tryGetTable(table_name, context); buffers[table_name] = std::make_shared(); buffers[table_name]->data = storage->getSampleBlockNonMaterialized(); @@ -207,8 +207,8 @@ void EventConsumer::onEvent(const BinlogEventPtr & receive_event, const MySQLRep void EventConsumer::fillSignColumnsAndMayFlush(Block & data, Int8 sign_value, UInt64 version_value, size_t fill_size, size_t prev_bytes) { - MutableColumnPtr sign_mutable_column = (*std::move(data.getByPosition(data.columns() - 2)).column).mutate(); - MutableColumnPtr version_mutable_column = (*std::move(data.getByPosition(data.columns() - 1)).column).mutate(); + MutableColumnPtr sign_mutable_column = IColumn::mutate(std::move(data.getByPosition(data.columns() - 2).column)); + MutableColumnPtr version_mutable_column = IColumn::mutate(std::move(data.getByPosition(data.columns() - 1).column)); ColumnInt8::Container & sign_column_data = assert_cast(*sign_mutable_column).getData(); ColumnUInt64::Container & version_column_data = assert_cast(*version_mutable_column).getData(); From 4a99ca0d80b6e40e295a191c3e7a6772717a5917 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 22 Jun 2020 11:12:05 +0800 Subject: [PATCH 070/374] ISSUES-4006 refactor --- src/Common/CurrentMetrics.cpp | 1 + src/Core/MySQLClient.cpp | 37 +- src/Core/MySQLClient.h | 8 +- src/Databases/DatabaseFactory.cpp | 4 +- .../MySQL/DatabaseMaterializeMySQL.cpp | 325 +++++++------ .../MySQL/DatabaseMaterializeMySQL.h | 100 ++-- .../MySQL/DatabaseMaterializeMySQLWrap.cpp | 188 -------- .../MySQL/DatabaseMaterializeMySQLWrap.h | 70 --- src/Databases/MySQL/EventConsumer.cpp | 261 ---------- src/Databases/MySQL/EventConsumer.h | 63 --- ...tings.cpp => MaterializeMySQLSettings.cpp} | 6 +- ...eSettings.h => MaterializeMySQLSettings.h} | 4 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 444 ++++++++++++++++++ .../MySQL/MaterializeMySQLSyncThread.h | 91 ++++ 14 files changed, 798 insertions(+), 804 deletions(-) delete mode 100644 src/Databases/MySQL/DatabaseMaterializeMySQLWrap.cpp delete mode 100644 src/Databases/MySQL/DatabaseMaterializeMySQLWrap.h delete mode 100644 src/Databases/MySQL/EventConsumer.cpp delete mode 100644 src/Databases/MySQL/EventConsumer.h rename src/Databases/MySQL/{MaterializeModeSettings.cpp => MaterializeMySQLSettings.cpp} (79%) rename src/Databases/MySQL/{MaterializeModeSettings.h => MaterializeMySQLSettings.h} (76%) create mode 100644 src/Databases/MySQL/MaterializeMySQLSyncThread.cpp create mode 100644 src/Databases/MySQL/MaterializeMySQLSyncThread.h diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 4bab9ef2844..afbf14b4a38 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -13,6 +13,7 @@ M(BackgroundMovePoolTask, "Number of active tasks in BackgroundProcessingPool for moves") \ M(BackgroundSchedulePoolTask, "Number of active tasks in BackgroundSchedulePool. This pool is used for periodic ReplicatedMergeTree tasks, like cleaning old data parts, altering data parts, replica re-initialization, etc.") \ M(BackgroundBufferFlushSchedulePoolTask, "Number of active tasks in BackgroundBufferFlushSchedulePool. This pool is used for periodic Buffer flushes") \ + M(BackgroundMySQLSyncSchedulePoolTask, "Number of active tasks in BackgroundMySQLSyncSchedulePoolTask. This pool is used for MySQL Materialize Database sync.") \ M(BackgroundDistributedSchedulePoolTask, "Number of active tasks in BackgroundDistributedSchedulePool. This pool is used for distributed sends that is done in background.") \ M(CacheDictionaryUpdateQueueBatches, "Number of 'batches' (a set of keys) in update queue in CacheDictionaries.") \ M(CacheDictionaryUpdateQueueKeys, "Exact number of keys in update queue in CacheDictionaries.") \ diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 5905de11dd2..109e8e1a7da 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -125,6 +125,9 @@ void MySQLClient::ping() void MySQLClient::startBinlogDump(UInt32 slave_id, String replicate_db, String binlog_file_name, UInt64 binlog_pos) { + if (dump_thread) + return; + /// Set binlog checksum to CRC32. String checksum = "CRC32"; writeCommand(Command::COM_QUERY, "SET @master_binlog_checksum = '" + checksum + "'"); @@ -142,15 +145,33 @@ void MySQLClient::startBinlogDump(UInt32 slave_id, String replicate_db, String b binlog_pos = binlog_pos < 4 ? 4 : binlog_pos; BinlogDump binlog_dump(binlog_pos, binlog_file_name, slave_id); packet_sender->sendPacket(binlog_dump, true); -} - -BinlogEventPtr MySQLClient::readOneBinlogEvent() -{ - while (true) + dump_thread.emplace([this]() { - packet_sender->receivePacket(replication); - return replication.readOneEvent(); - } + while (true) + { + try + { + packet_sender->receivePacket(replication); + events.push(std::make_pair(replication.readOneEvent(), replication.getPosition())); + } + catch(...) + { + tryLogCurrentException("MySQLClient"); + /// TODO: maybe sleep? + } + } + }); +} + +BinlogEventPtr MySQLClient::readOneBinlogEvent(UInt64 milliseconds) +{ + std::pair event; + + if (!events.tryPop(event, milliseconds)) + return {}; + + last_position = event.second; + return event.first; } } diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 5c42e5c5d34..753e2a6cd44 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -35,8 +36,8 @@ public: void ping(); void startBinlogDump(UInt32 slave_id, String replicate_db, String binlog_file_name, UInt64 binlog_pos); - BinlogEventPtr readOneBinlogEvent(); - Position getPosition() const { return replication.getPosition(); } + BinlogEventPtr readOneBinlogEvent(UInt64 milliseconds = 0); + Position getPosition() const { return last_position; } private: String host; @@ -58,6 +59,9 @@ private: std::unique_ptr socket; std::optional address; std::shared_ptr packet_sender; + Position last_position; + std::optional dump_thread; + ConcurrentBoundedQueue> events{1}; void handshake(); void registerSlaveOnMaster(UInt32 slave_id); diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index e7f8b6f22de..71fdc15f70f 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -20,8 +20,8 @@ #if USE_MYSQL # include # include +# include # include -# include # include # include # include @@ -119,7 +119,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String const auto & [remote_host_name, remote_port] = parseAddress(host_name_and_port, 3306); auto mysql_pool = mysqlxx::Pool(mysql_database_name, remote_host_name, mysql_user_name, mysql_user_password, remote_port); - auto materialize_mode_settings = std::make_unique(); + auto materialize_mode_settings = std::make_unique(); if (engine_define->settings) materialize_mode_settings->loadFromQuery(*engine_define); diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index b3d55c46d66..32b2e7c3b40 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -1,198 +1,195 @@ -#if !defined(ARCADIA_BUILD) -# include "config_core.h" -#endif - -#if USE_MYSQL - #include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include namespace DB { +static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; + namespace ErrorCodes { - extern const int INCORRECT_QUERY; -} - -static inline BlockIO tryToExecuteQuery(const String & query_to_execute, const Context & context_, const String & comment) -{ - try - { - Context context = context_; - context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - context.setCurrentQueryId(""); // generate random query_id - return executeQuery("/*" + comment + "*/ " + query_to_execute, context, true); - } - catch (...) - { - tryLogCurrentException("DatabaseMaterializeMySQL", "Query " + query_to_execute + " wasn't finished successfully"); - throw; - } - - LOG_DEBUG(&Logger::get("DatabaseMaterializeMySQL"), "Executed query: " << query_to_execute); + extern const int NOT_IMPLEMENTED; } DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( - const Context & context, const String & database_name_, const String & metadata_path_ - , const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_ - , MySQLClient && client_ , std::unique_ptr settings_) - : DatabaseMaterializeMySQLWrap(std::make_shared(database_name_, metadata_path_, context), database_engine_define_->clone(), "DatabaseMaterializeMySQL") - , global_context(context.getGlobalContext()), metadata_path(metadata_path_), mysql_database_name(mysql_database_name_) - , pool(std::move(pool_)), client(std::move(client_)), settings(std::move(settings_)) + const Context & context, const String & database_name_, const String & metadata_path_, const IAST * database_engine_define_ + , const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, std::unique_ptr settings_) + : IDatabase(database_name_), engine_define(database_engine_define_->clone()) + , nested_database(std::make_shared(database_name_, metadata_path_, context)) + , settings(std::move(settings_)), log(&Poco::Logger::get("DatabaseMaterializeMySQL")) + , materialize_thread(context, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), settings.get()) { - /// TODO: 做简单的check, 失败即报错 - scheduleSynchronized(); } -BlockOutputStreamPtr DatabaseMaterializeMySQL::getTableOutput(const String & table_name) +void DatabaseMaterializeMySQL::setException(const std::exception_ptr & exception_) { - String with_database_table_name = backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name); - BlockIO res = tryToExecuteQuery("INSERT INTO " + with_database_table_name + " VALUES", global_context, ""); - - if (!res.out) - throw Exception("LOGICAL ERROR:", ErrorCodes::LOGICAL_ERROR); - - return res.out; + std::unique_lock lock(mutex); + exception = exception_; } -void DatabaseMaterializeMySQL::cleanOutdatedTables() +DatabasePtr DatabaseMaterializeMySQL::getNestedDatabase() const { - auto ddl_guard = DatabaseCatalog::instance().getDDLGuard(database_name, ""); - const DatabasePtr & clean_database = DatabaseCatalog::instance().getDatabase(database_name); + std::unique_lock lock(mutex); - for (auto iterator = clean_database->getTablesIterator(); iterator->isValid(); iterator->next()) - { - String table = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(iterator->name()); - String comment = String("Clean ") + table + " for dump mysql."; - tryToExecuteQuery("DROP TABLE " + table, global_context, comment); - } + if (exception) + std::rethrow_exception(exception); + + return nested_database; } -void DatabaseMaterializeMySQL::dumpDataForTables(mysqlxx::Pool::Entry & connection, MaterializeMetadata & master_info, const std::function & is_cancelled) +ASTPtr DatabaseMaterializeMySQL::getCreateDatabaseQuery() const { - auto iterator = master_info.need_dumping_tables.begin(); - for (; iterator != master_info.need_dumping_tables.end() && !is_cancelled(); ++iterator) - { - const auto & table_name = iterator->first; - MySQLTableStruct table_struct = visitCreateQuery(iterator->second, global_context, database_name); - String comment = String("Dumping ") + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name); - tryToExecuteQuery(toCreateQuery(table_struct, global_context), global_context, comment); - - BlockOutputStreamPtr out = std::make_shared(master_info.version, getTableOutput(table_name)); - MySQLBlockInputStream input(connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name), out->getHeader(), DEFAULT_BLOCK_SIZE); - copyData(input, *out, is_cancelled); - } + const auto & create_query = std::make_shared(); + create_query->database = database_name; + create_query->set(create_query->storage, engine_define); + return create_query; } - -std::optional DatabaseMaterializeMySQL::prepareSynchronized(std::unique_lock & lock, const std::function & is_cancelled) -{ - while (!is_cancelled()) - { - try - { - LOG_DEBUG(log, "Checking " + database_name + " database status."); - while (!is_cancelled && !DatabaseCatalog::instance().isDatabaseExist(database_name)) - sync_cond.wait_for(lock, std::chrono::seconds(1)); - - LOG_DEBUG(log, database_name + " database status is OK."); - - mysqlxx::PoolWithFailover::Entry connection = pool.get(); - MaterializeMetadata metadata(connection, getMetadataPath() + "/.metadata", mysql_database_name); - - if (!metadata.need_dumping_tables.empty()) - { - metadata.transaction(Position(metadata.binlog_position, metadata.binlog_file), [&]() - { - cleanOutdatedTables(); - dumpDataForTables(connection, metadata, is_cancelled); - }); - } - - client.connect(); - client.startBinlogDump(std::rand(), mysql_database_name, metadata.binlog_file, metadata.binlog_position); - return metadata; - } - catch (mysqlxx::Exception & ) - { - tryLogCurrentException(log); - - /// Avoid busy loop when MySQL is not available. - sleepForMilliseconds(settings->max_wait_time_when_mysql_unavailable); - } - } - - return {}; -} - -void DatabaseMaterializeMySQL::scheduleSynchronized() -{ - background_thread_pool.scheduleOrThrowOnError([&]() - { - ThreadStatus thread_status; - setThreadName("MySQLDBSync"); - - std::unique_lock lock(sync_mutex); - const auto quit_requested = [this] { return sync_quit.load(std::memory_order_relaxed); }; - - try - { - std::optional metadata = prepareSynchronized(lock, quit_requested); - - if (!quit_requested() && metadata) - { - EventConsumer consumer(getDatabaseName(), global_context, *metadata, *settings); - - while (!quit_requested()) - { - const auto & event = client.readOneBinlogEvent(); - consumer.onEvent(event, client.getPosition()); - } - } - } - catch(...) - { - setException(std::current_exception()); - } - }); -} -DatabaseMaterializeMySQL::~DatabaseMaterializeMySQL() +void DatabaseMaterializeMySQL::loadStoredObjects(Context & context, bool has_force_restore_data_flag) { try { - if (!sync_quit) - { - { - sync_quit = true; - std::lock_guard lock(sync_mutex); - } - - sync_cond.notify_one(); - background_thread_pool.wait(); - } + LOG_DEBUG(log, "Loading MySQL nested database stored objects."); + getNestedDatabase()->loadStoredObjects(context, has_force_restore_data_flag); + LOG_DEBUG(log, "Loaded MySQL nested database stored objects."); } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(log, "Cannot load MySQL nested database stored objects."); + throw; } } +void DatabaseMaterializeMySQL::shutdown() +{ + getNestedDatabase()->shutdown(); } -#endif +bool DatabaseMaterializeMySQL::empty() const +{ + return getNestedDatabase()->empty(); +} + +String DatabaseMaterializeMySQL::getDataPath() const +{ + return getNestedDatabase()->getDataPath(); +} + +String DatabaseMaterializeMySQL::getMetadataPath() const +{ + return getNestedDatabase()->getMetadataPath(); +} + +String DatabaseMaterializeMySQL::getTableDataPath(const String & table_name) const +{ + return getNestedDatabase()->getTableDataPath(table_name); +} + +String DatabaseMaterializeMySQL::getTableDataPath(const ASTCreateQuery & query) const +{ + return getNestedDatabase()->getTableDataPath(query); +} + +String DatabaseMaterializeMySQL::getObjectMetadataPath(const String & table_name) const +{ + return getNestedDatabase()->getObjectMetadataPath(table_name); +} + +UUID DatabaseMaterializeMySQL::tryGetTableUUID(const String & table_name) const +{ + return getNestedDatabase()->tryGetTableUUID(table_name); +} + +time_t DatabaseMaterializeMySQL::getObjectMetadataModificationTime(const String & name) const +{ + return getNestedDatabase()->getObjectMetadataModificationTime(name); +} + +void DatabaseMaterializeMySQL::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + throw Exception("MySQL database in locality_data mode does not support create table.", ErrorCodes::NOT_IMPLEMENTED); + + getNestedDatabase()->createTable(context, name, table, query); +} + +void DatabaseMaterializeMySQL::dropTable(const Context & context, const String & name, bool no_delay) +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + throw Exception("MySQL database in locality_data mode does not support drop table.", ErrorCodes::NOT_IMPLEMENTED); + + getNestedDatabase()->dropTable(context, name, no_delay); +} + +void DatabaseMaterializeMySQL::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + throw Exception("MySQL database in locality_data mode does not support attach table.", ErrorCodes::NOT_IMPLEMENTED); + + getNestedDatabase()->attachTable(name, table, relative_table_path); +} + +StoragePtr DatabaseMaterializeMySQL::detachTable(const String & name) +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + throw Exception("MySQL database in locality_data mode does not support detach table.", ErrorCodes::NOT_IMPLEMENTED); + + return getNestedDatabase()->detachTable(name); +} + +void DatabaseMaterializeMySQL::renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange) +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + throw Exception("MySQL database in locality_data mode does not support rename table.", ErrorCodes::NOT_IMPLEMENTED); + + getNestedDatabase()->renameTable(context, name, to_database, to_name, exchange); +} + +void DatabaseMaterializeMySQL::alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + throw Exception("MySQL database in locality_data mode does not support alter table.", ErrorCodes::NOT_IMPLEMENTED); + + getNestedDatabase()->alterTable(context, table_id, metadata); +} + +bool DatabaseMaterializeMySQL::shouldBeEmptyOnDetach() const +{ + return getNestedDatabase()->shouldBeEmptyOnDetach(); +} + +void DatabaseMaterializeMySQL::drop(const Context & context) +{ + getNestedDatabase()->drop(context); +} + +bool DatabaseMaterializeMySQL::isTableExist(const String & name, const Context & context) const +{ + return getNestedDatabase()->isTableExist(name, context); +} + +StoragePtr DatabaseMaterializeMySQL::tryGetTable(const String & name, const Context & context) const +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + return std::make_shared(getNestedDatabase()->tryGetTable(name, context)); + + return getNestedDatabase()->tryGetTable(name, context); +} + +DatabaseTablesIteratorPtr DatabaseMaterializeMySQL::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) +{ + if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + { + DatabaseTablesIteratorPtr iterator = getNestedDatabase()->getTablesIterator(context, filter_by_table_name); + return std::make_unique(std::move(iterator)); + } + + return getNestedDatabase()->getTablesIterator(context, filter_by_table_name); +} + +} diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h index 74526bb4635..e698bf67689 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -1,64 +1,80 @@ #pragma once -#include "config_core.h" - -#if USE_MYSQL - -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include namespace DB { -class DatabaseMaterializeMySQL : public DatabaseMaterializeMySQLWrap +class DatabaseMaterializeMySQL : public IDatabase { public: - ~DatabaseMaterializeMySQL() override; - DatabaseMaterializeMySQL( const Context & context, const String & database_name_, const String & metadata_path_, - const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_, - MySQLClient && client_, std::unique_ptr settings_); + const IAST * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_, + MySQLClient && client_, std::unique_ptr settings_); - String getEngineName() const override { return "MySQL"; } + void setException(const std::exception_ptr & exception); +protected: + ASTPtr engine_define; + DatabasePtr nested_database; + std::unique_ptr settings; -private: - const Context & global_context; - String metadata_path; - String mysql_database_name; + Poco::Logger * log; + MaterializeMySQLSyncThread materialize_thread; - mutable mysqlxx::Pool pool; - mutable MySQLClient client; - std::unique_ptr settings; + mutable std::mutex mutex; + std::exception_ptr exception; - void cleanOutdatedTables(); + DatabasePtr getNestedDatabase() const; - void scheduleSynchronized(); +public: + ASTPtr getCreateDatabaseQuery() const override; - BlockOutputStreamPtr getTableOutput(const String & table_name); + void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override; - std::optional prepareSynchronized(std::unique_lock & lock, const std::function & is_cancelled); + void shutdown() override; - void dumpDataForTables(mysqlxx::Pool::Entry & connection, MaterializeMetadata & master_info, const std::function & is_cancelled); + bool empty() const override; - std::mutex sync_mutex; - std::atomic sync_quit{false}; - std::condition_variable sync_cond; - ThreadPool background_thread_pool{1}; + String getDataPath() const override; + + String getTableDataPath(const String & table_name) const override; + + String getTableDataPath(const ASTCreateQuery & query) const override; + + UUID tryGetTableUUID(const String & table_name) const override; + + void createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) override; + + void dropTable(const Context & context, const String & name, bool no_delay) override; + + void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) override; + + StoragePtr detachTable(const String & name) override; + + void renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange) override; + + void alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; + + time_t getObjectMetadataModificationTime(const String & name) const override; + + String getMetadataPath() const override; + + String getObjectMetadataPath(const String & table_name) const override; + + bool shouldBeEmptyOnDetach() const override; + + void drop(const Context & context) override; + + bool isTableExist(const String & name, const Context & context) const override; + + StoragePtr tryGetTable(const String & name, const Context & context) const override; + + DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; }; } - -#endif diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQLWrap.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQLWrap.cpp deleted file mode 100644 index 6b628ddf26d..00000000000 --- a/src/Databases/MySQL/DatabaseMaterializeMySQLWrap.cpp +++ /dev/null @@ -1,188 +0,0 @@ -#include - -#include -#include -#include -#include - -namespace DB -{ - -static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -DatabaseMaterializeMySQLWrap::DatabaseMaterializeMySQLWrap(const DatabasePtr & nested_database_, const ASTPtr & database_engine_define_, const String & log_name) - : IDatabase(nested_database_->getDatabaseName()), nested_database(nested_database_), database_engine_define(database_engine_define_), log(&Logger::get(log_name)) -{ -} - -void DatabaseMaterializeMySQLWrap::setException(const std::exception_ptr & exception_) -{ - std::unique_lock lock(mutex); - exception = exception_; -} - -DatabasePtr DatabaseMaterializeMySQLWrap::getNestedDatabase() const -{ - std::unique_lock lock(mutex); - - if (exception) - std::rethrow_exception(exception); - - return nested_database; -} - -ASTPtr DatabaseMaterializeMySQLWrap::getCreateDatabaseQuery() const -{ - const auto & create_query = std::make_shared(); - create_query->database = database_name; - create_query->set(create_query->storage, database_engine_define); - return create_query; -} -void DatabaseMaterializeMySQLWrap::loadStoredObjects(Context & context, bool has_force_restore_data_flag) -{ - try - { - LOG_DEBUG(log, "Loading MySQL nested database stored objects."); - getNestedDatabase()->loadStoredObjects(context, has_force_restore_data_flag); - LOG_DEBUG(log, "Loaded MySQL nested database stored objects."); - } - catch (...) - { - tryLogCurrentException(log, "Cannot load MySQL nested database stored objects."); - throw; - } -} - -void DatabaseMaterializeMySQLWrap::shutdown() -{ - getNestedDatabase()->shutdown(); -} - -bool DatabaseMaterializeMySQLWrap::empty() const -{ - return getNestedDatabase()->empty(); -} - -String DatabaseMaterializeMySQLWrap::getDataPath() const -{ - return getNestedDatabase()->getDataPath(); -} - -String DatabaseMaterializeMySQLWrap::getMetadataPath() const -{ - return getNestedDatabase()->getMetadataPath(); -} - -String DatabaseMaterializeMySQLWrap::getTableDataPath(const String & table_name) const -{ - return getNestedDatabase()->getTableDataPath(table_name); -} - -String DatabaseMaterializeMySQLWrap::getTableDataPath(const ASTCreateQuery & query) const -{ - return getNestedDatabase()->getTableDataPath(query); -} - -String DatabaseMaterializeMySQLWrap::getObjectMetadataPath(const String & table_name) const -{ - return getNestedDatabase()->getObjectMetadataPath(table_name); -} - -UUID DatabaseMaterializeMySQLWrap::tryGetTableUUID(const String & table_name) const -{ - return getNestedDatabase()->tryGetTableUUID(table_name); -} - -time_t DatabaseMaterializeMySQLWrap::getObjectMetadataModificationTime(const String & name) const -{ - return getNestedDatabase()->getObjectMetadataModificationTime(name); -} - -void DatabaseMaterializeMySQLWrap::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) -{ - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) - throw Exception("MySQL database in locality_data mode does not support create table.", ErrorCodes::NOT_IMPLEMENTED); - - getNestedDatabase()->createTable(context, name, table, query); -} - -void DatabaseMaterializeMySQLWrap::dropTable(const Context & context, const String & name, bool no_delay) -{ - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) - throw Exception("MySQL database in locality_data mode does not support drop table.", ErrorCodes::NOT_IMPLEMENTED); - - getNestedDatabase()->dropTable(context, name, no_delay); -} - -void DatabaseMaterializeMySQLWrap::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) -{ - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) - throw Exception("MySQL database in locality_data mode does not support attach table.", ErrorCodes::NOT_IMPLEMENTED); - - getNestedDatabase()->attachTable(name, table, relative_table_path); -} - -StoragePtr DatabaseMaterializeMySQLWrap::detachTable(const String & name) -{ - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) - throw Exception("MySQL database in locality_data mode does not support detach table.", ErrorCodes::NOT_IMPLEMENTED); - - return getNestedDatabase()->detachTable(name); -} - -void DatabaseMaterializeMySQLWrap::renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange) -{ - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) - throw Exception("MySQL database in locality_data mode does not support rename table.", ErrorCodes::NOT_IMPLEMENTED); - - getNestedDatabase()->renameTable(context, name, to_database, to_name, exchange); -} - -void DatabaseMaterializeMySQLWrap::alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) -{ - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) - throw Exception("MySQL database in locality_data mode does not support alter table.", ErrorCodes::NOT_IMPLEMENTED); - - getNestedDatabase()->alterTable(context, table_id, metadata); -} - -bool DatabaseMaterializeMySQLWrap::shouldBeEmptyOnDetach() const -{ - return getNestedDatabase()->shouldBeEmptyOnDetach(); -} - -void DatabaseMaterializeMySQLWrap::drop(const Context & context) -{ - getNestedDatabase()->drop(context); -} - -bool DatabaseMaterializeMySQLWrap::isTableExist(const String & name) const -{ - return getNestedDatabase()->isTableExist(name); -} - -StoragePtr DatabaseMaterializeMySQLWrap::tryGetTable(const String & name) const -{ - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) - return std::make_shared(getNestedDatabase()->tryGetTable(name)); - - return getNestedDatabase()->tryGetTable(name); -} - -DatabaseTablesIteratorPtr DatabaseMaterializeMySQLWrap::getTablesIterator(const FilterByNameFunction & filter_by_table_name) -{ - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) - { - DatabaseTablesIteratorPtr iterator = getNestedDatabase()->getTablesIterator(filter_by_table_name); - return std::make_unique(std::move(iterator)); - } - - return getNestedDatabase()->getTablesIterator(filter_by_table_name); -} - -} diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQLWrap.h b/src/Databases/MySQL/DatabaseMaterializeMySQLWrap.h deleted file mode 100644 index 0f16661c66f..00000000000 --- a/src/Databases/MySQL/DatabaseMaterializeMySQLWrap.h +++ /dev/null @@ -1,70 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class DatabaseMaterializeMySQLWrap : public IDatabase -{ -public: - ASTPtr getCreateDatabaseQuery() const override; - - void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override; - - DatabaseMaterializeMySQLWrap(const DatabasePtr & nested_database_, const ASTPtr & database_engine_define_, const String & log_name); -protected: - DatabasePtr nested_database; - ASTPtr database_engine_define; - Poco::Logger * log; - - mutable std::mutex mutex; - std::exception_ptr exception; - - DatabasePtr getNestedDatabase() const; - - void setException(const std::exception_ptr & exception); - -public: - void shutdown() override; - - bool empty() const override; - - String getDataPath() const override; - - String getTableDataPath(const String & table_name) const override; - - String getTableDataPath(const ASTCreateQuery & query) const override; - - UUID tryGetTableUUID(const String & table_name) const override; - - void createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) override; - - void dropTable(const Context & context, const String & name, bool no_delay) override; - - void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) override; - - StoragePtr detachTable(const String & name) override; - - void renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange) override; - - void alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; - - time_t getObjectMetadataModificationTime(const String & name) const override; - - String getMetadataPath() const override; - - String getObjectMetadataPath(const String & table_name) const override; - - bool shouldBeEmptyOnDetach() const override; - - void drop(const Context & context) override; - - bool isTableExist(const String & name) const override; - - StoragePtr tryGetTable(const String & name) const override; - - DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; -}; - -} diff --git a/src/Databases/MySQL/EventConsumer.cpp b/src/Databases/MySQL/EventConsumer.cpp deleted file mode 100644 index 2054f5810ef..00000000000 --- a/src/Databases/MySQL/EventConsumer.cpp +++ /dev/null @@ -1,261 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -using namespace MySQLReplication; - -EventConsumer::~EventConsumer() -{ - if (!quit && !background_exception) - { - { - quit = true; - std::lock_guard lock(mutex); - } - - cond.notify_one(); - background_thread_pool.wait(); - } -} - -EventConsumer::EventConsumer( - const String & database_, const Context & context_, MaterializeMetadata & metadata_, MaterializeModeSettings & settings_) - : metadata(metadata_), context(context_), settings(settings_), database(database_), prev_version(metadata.version) -{ - background_thread_pool.scheduleOrThrowOnError([&]() - { - ThreadStatus thread_status; - setThreadName("MySQLDBSync"); - std::unique_lock lock(mutex); - const auto quit_requested = [this] { return quit.load(std::memory_order_relaxed); }; - - while (!quit_requested() && !background_exception) - { - if (!buffers.empty() && total_bytes_in_buffers) - flushBuffers(); - - cond.wait_for(lock, std::chrono::milliseconds(settings.max_flush_data_time), quit_requested); - } - }); -} - -void EventConsumer::onWriteData(const String & table_name, const std::vector & rows_data) -{ - BufferPtr buffer = getTableBuffer(table_name); - - size_t prev_bytes = buffer->data.bytes(); - for (size_t column = 0; column < buffer->data.columns() - 2; ++column) - { - MutableColumnPtr col_to = IColumn::mutate(std::move(buffer->data.getByPosition(column).column)); - - for (size_t index = 0; index < rows_data.size(); ++index) - col_to->insert(DB::get(rows_data[index])[column]); - } - - fillSignColumnsAndMayFlush(buffer->data, 1, ++metadata.version, rows_data.size(), prev_bytes); -} - -static inline bool differenceSortingKeys(const Tuple & row_old_data, const Tuple & row_new_data, const std::vector sorting_columns_index) -{ - for (const auto & sorting_column_index : sorting_columns_index) - if (row_old_data[sorting_column_index] != row_new_data[sorting_column_index]) - return true; - - return false; -} - -void EventConsumer::onUpdateData(const String & table_name, const std::vector & rows_data) -{ - if (rows_data.size() % 2 != 0) - throw Exception("LOGICAL ERROR: ", ErrorCodes::LOGICAL_ERROR); - - BufferPtr buffer = getTableBuffer(table_name); - - size_t prev_bytes = buffer->data.bytes(); - std::vector difference_sorting_keys_mark(rows_data.size() / 2); - - for (size_t index = 0; index < rows_data.size(); index += 2) - difference_sorting_keys_mark.emplace_back(differenceSortingKeys( - DB::get(rows_data[index]), DB::get(rows_data[index + 1]), buffer->sorting_columns_index)); - - for (size_t column = 0; column < buffer->data.columns() - 2; ++column) - { - MutableColumnPtr col_to = IColumn::mutate(std::move(buffer->data.getByPosition(column).column)); - - for (size_t index = 0; index < rows_data.size(); index += 2) - { - if (likely(!difference_sorting_keys_mark[index / 2])) - col_to->insert(DB::get(rows_data[index + 1])[column]); - else - { - /// If the sorting keys is modified, we should cancel the old data, but this should not happen frequently - col_to->insert(DB::get(rows_data[index])[column]); - col_to->insert(DB::get(rows_data[index + 1])[column]); - } - } - } - - MutableColumnPtr sign_mutable_column = IColumn::mutate(std::move(buffer->data.getByPosition(buffer->data.columns() - 2).column)); - MutableColumnPtr version_mutable_column = IColumn::mutate(std::move(buffer->data.getByPosition(buffer->data.columns() - 1).column)); - - ColumnInt8::Container & sign_column_data = assert_cast(*sign_mutable_column).getData(); - ColumnUInt64::Container & version_column_data = assert_cast(*version_mutable_column).getData(); - - UInt64 new_version = ++metadata.version; - for (size_t index = 0; index < rows_data.size(); index += 2) - { - if (likely(!difference_sorting_keys_mark[index / 2])) - { - sign_column_data.emplace_back(1); - version_column_data.emplace_back(new_version); - } - else - { - /// If the sorting keys is modified, we should cancel the old data, but this should not happen frequently - sign_column_data.emplace_back(-1); - sign_column_data.emplace_back(1); - version_column_data.emplace_back(new_version); - version_column_data.emplace_back(new_version); - } - } - - total_bytes_in_buffers += (buffer->data.bytes() - prev_bytes); - if (buffer->data.rows() >= settings.max_rows_in_buffer || total_bytes_in_buffers >= settings.max_bytes_in_buffers) - flushBuffers(); -} - -void EventConsumer::onDeleteData(const String & table_name, const std::vector & rows_data) -{ - BufferPtr buffer = getTableBuffer(table_name); - - size_t prev_bytes = buffer->data.bytes(); - for (size_t column = 0; column < buffer->data.columns() - 2; ++column) - { - MutableColumnPtr col_to = IColumn::mutate(std::move(buffer->data.getByPosition(column).column)); - - for (size_t index = 0; index < rows_data.size(); ++index) - col_to->insert(DB::get(rows_data[index])[column]); - } - - fillSignColumnsAndMayFlush(buffer->data, -1, ++metadata.version, rows_data.size(), prev_bytes); -} - -EventConsumer::BufferPtr EventConsumer::getTableBuffer(const String & table_name) -{ - if (buffers.find(table_name) == buffers.end()) - { - StoragePtr storage = DatabaseCatalog::instance().getDatabase(database)->tryGetTable(table_name, context); - - buffers[table_name] = std::make_shared(); - buffers[table_name]->data = storage->getSampleBlockNonMaterialized(); - if (StorageMergeTree * table_merge_tree = dynamic_cast(storage.get())) - { - Names required_for_sorting_key = table_merge_tree->getColumnsRequiredForSortingKey(); - - for (const auto & required_name_for_sorting_key : required_for_sorting_key) - buffers[table_name]->sorting_columns_index.emplace_back( - buffers[table_name]->data.getPositionByName(required_name_for_sorting_key)); - } - } - - return buffers[table_name]; -} - -void EventConsumer::onEvent(const BinlogEventPtr & receive_event, const MySQLReplication::Position & position) -{ - std::unique_lock lock(mutex); - - if (background_exception) - background_thread_pool.wait(); - - last_position = position; - if (receive_event->type() == MYSQL_WRITE_ROWS_EVENT) - { - WriteRowsEvent & write_rows_event = static_cast(*receive_event); - write_rows_event.dump(); - onWriteData(write_rows_event.table, write_rows_event.rows); - } - else if (receive_event->type() == MYSQL_UPDATE_ROWS_EVENT) - { - UpdateRowsEvent & update_rows_event = static_cast(*receive_event); - update_rows_event.dump(); - onUpdateData(update_rows_event.table, update_rows_event.rows); - } - else if (receive_event->type() == MYSQL_DELETE_ROWS_EVENT) - { - DeleteRowsEvent & delete_rows_event = static_cast(*receive_event); - delete_rows_event.dump(); - onDeleteData(delete_rows_event.table, delete_rows_event.rows); - } - else if (receive_event->type() == MYSQL_QUERY_EVENT) - { - /// TODO: 识别, 查看是否支持的DDL, 支持的话立即刷新当前的数据, 然后执行DDL. -// flush_function(); - /// TODO: 直接使用Interpreter执行即可 - } -} - -void EventConsumer::fillSignColumnsAndMayFlush(Block & data, Int8 sign_value, UInt64 version_value, size_t fill_size, size_t prev_bytes) -{ - MutableColumnPtr sign_mutable_column = IColumn::mutate(std::move(data.getByPosition(data.columns() - 2).column)); - MutableColumnPtr version_mutable_column = IColumn::mutate(std::move(data.getByPosition(data.columns() - 1).column)); - - ColumnInt8::Container & sign_column_data = assert_cast(*sign_mutable_column).getData(); - ColumnUInt64::Container & version_column_data = assert_cast(*version_mutable_column).getData(); - - for (size_t index = 0; index < fill_size; ++index) - { - sign_column_data.emplace_back(sign_value); - version_column_data.emplace_back(version_value); - } - - total_bytes_in_buffers += (data.bytes() - prev_bytes); - if (data.rows() >= settings.max_rows_in_buffer || total_bytes_in_buffers >= settings.max_bytes_in_buffers) - flushBuffers(); -} - -void EventConsumer::flushBuffers() -{ - /// TODO: 事务保证 - try - { - for (auto & table_name_and_buffer : buffers) - { - const String & table_name = table_name_and_buffer.first; - BufferPtr & buffer = table_name_and_buffer.second; - - Context query_context = context; - query_context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - query_context.setCurrentQueryId(""); // generate random query_id - String with_database_table_name = backQuoteIfNeed(database) + "." + backQuoteIfNeed(table_name); - BlockIO res = executeQuery("INSERT INTO " + with_database_table_name + " VALUES", query_context, true); - - OneBlockInputStream input(buffer->data); - copyData(input, *res.out); - } - - buffers.clear(); - total_bytes_in_buffers = 0; - prev_version = metadata.version; - } - catch(...) - { - buffers.clear(); - total_bytes_in_buffers = 0; - metadata.version = prev_version; - background_exception = true; - throw; - } -} - -} diff --git a/src/Databases/MySQL/EventConsumer.h b/src/Databases/MySQL/EventConsumer.h deleted file mode 100644 index f21265efe9d..00000000000 --- a/src/Databases/MySQL/EventConsumer.h +++ /dev/null @@ -1,63 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include "MaterializeMetadata.h" - -namespace DB -{ - -class EventConsumer : private boost::noncopyable -{ -public: - ~EventConsumer(); - - void onEvent(const MySQLReplication::BinlogEventPtr & event, const MySQLReplication::Position & position); - - EventConsumer(const String & database_, const Context & context, MaterializeMetadata & metadata_, MaterializeModeSettings & settings_); -private: - MaterializeMetadata & metadata; - - const Context & context; - const MaterializeModeSettings & settings; - - String database; - size_t prev_version; - size_t total_bytes_in_buffers = 0; - MySQLReplication::Position last_position; - - struct Buffer - { - Block data; - std::vector sorting_columns_index; - }; - - using BufferPtr = std::shared_ptr; - std::unordered_map buffers; - - void flushBuffers(); - - BufferPtr getTableBuffer(const String & table_name); - - void onWriteData(const std::string & table_name, const std::vector & rows_data); - - void onUpdateData(const std::string & table_name, const std::vector & rows_data); - - void onDeleteData(const std::string & table_name, const std::vector & rows_data); - - void fillSignColumnsAndMayFlush(Block & data, Int8 sign_value, UInt64 version_value, size_t fill_size, size_t prev_bytes); - - mutable std::mutex mutex; - std::condition_variable cond; - std::atomic_bool quit = false; - std::atomic_bool background_exception = false; - ThreadPool background_thread_pool{1}; -}; - -} diff --git a/src/Databases/MySQL/MaterializeModeSettings.cpp b/src/Databases/MySQL/MaterializeMySQLSettings.cpp similarity index 79% rename from src/Databases/MySQL/MaterializeModeSettings.cpp rename to src/Databases/MySQL/MaterializeMySQLSettings.cpp index d8a25fa3bb9..fba4eaae2fa 100644 --- a/src/Databases/MySQL/MaterializeModeSettings.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSettings.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -13,9 +13,9 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -IMPLEMENT_SETTINGS_COLLECTION(MaterializeModeSettings, LIST_OF_MATERIALIZE_MODE_SETTINGS) +IMPLEMENT_SETTINGS_COLLECTION(MaterializeMySQLSettings, LIST_OF_MATERIALIZE_MODE_SETTINGS) -void MaterializeModeSettings::loadFromQuery(ASTStorage & storage_def) +void MaterializeMySQLSettings::loadFromQuery(ASTStorage & storage_def) { if (storage_def.settings) { diff --git a/src/Databases/MySQL/MaterializeModeSettings.h b/src/Databases/MySQL/MaterializeMySQLSettings.h similarity index 76% rename from src/Databases/MySQL/MaterializeModeSettings.h rename to src/Databases/MySQL/MaterializeMySQLSettings.h index 35c65253c87..77e8514d473 100644 --- a/src/Databases/MySQL/MaterializeModeSettings.h +++ b/src/Databases/MySQL/MaterializeMySQLSettings.h @@ -11,11 +11,13 @@ class ASTStorage; /** Settings for the MySQL Database engine(materialize mode). * Could be loaded from a CREATE DATABASE query (SETTINGS clause). */ -struct MaterializeModeSettings : public SettingsCollection +struct MaterializeMySQLSettings : public SettingsCollection { #define LIST_OF_MATERIALIZE_MODE_SETTINGS(M) \ M(SettingBool, locality_data, false, "", 0) \ M(SettingUInt64, max_rows_in_buffer, DEFAULT_BLOCK_SIZE, "", 0) \ + M(SettingUInt64, max_bytes_in_buffer, DBMS_DEFAULT_BUFFER_SIZE, "", 0) \ + M(SettingUInt64, max_rows_in_buffers, DEFAULT_BLOCK_SIZE, "", 0) \ M(SettingUInt64, max_bytes_in_buffers, DBMS_DEFAULT_BUFFER_SIZE, "", 0) \ M(SettingUInt64, max_flush_data_time, 1000, "", 0) \ M(SettingUInt64, max_wait_time_when_mysql_unavailable, 1000, "", 0) \ diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp new file mode 100644 index 00000000000..83f7b45ed28 --- /dev/null +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -0,0 +1,444 @@ +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#if USE_MYSQL + +#include + +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + +namespace CurrentMetrics +{ + extern const Metric MemoryTracking; + extern const Metric BackgroundMySQLSyncSchedulePoolTask; +} + +static BlockIO tryToExecuteQuery(const String & query_to_execute, const Context & context_, const String & comment) +{ + try + { + Context context = context_; + context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + context.setCurrentQueryId(""); // generate random query_id + return executeQuery("/*" + comment + "*/ " + query_to_execute, context, true); + } + catch (...) + { + tryLogCurrentException("MaterializeMySQLSyncThread", "Query " + query_to_execute + " wasn't finished successfully"); + throw; + } + +// LOG_DEBUG(&Logger::get("MaterializeMySQLSyncThread"), "Executed query: " + query_to_execute); +} + +static inline DatabaseMaterializeMySQL & getDatabase(const String & database_name) +{ + DatabasePtr database = DatabaseCatalog::instance().getDatabase(database_name); + + if (DatabaseMaterializeMySQL * database_materialize = typeid_cast(database.get())) + return *database_materialize; + + throw Exception("", ErrorCodes::LOGICAL_ERROR); +} + +MaterializeMySQLSyncThread::~MaterializeMySQLSyncThread() +{ + try + { + if (!sync_quit) + { + { + sync_quit = true; + std::lock_guard lock(sync_mutex); + } + + sync_cond.notify_one(); +// sync_task_handler->deactivate(); +// flush_task_handler->deactivate(); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +MaterializeMySQLSyncThread::MaterializeMySQLSyncThread( + const Context & context, const String & database_name_, const String & mysql_database_name_, + mysqlxx::Pool && pool_, MySQLClient && client_, MaterializeMySQLSettings * settings_) + : global_context(context), database_name(database_name_), mysql_database_name(mysql_database_name_) + , pool(std::move(pool_)), client(std::move(client_)), settings(settings_) +{ + /// TODO: 做简单的check, 失败即报错 + startSynchronization(); +} + +/*MaterializeMySQLSyncThread::MaterializeMySQLSyncThread( + const Context & context, const String & database_name_, const String & metadata_path_ + , const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_ + , MySQLClient && client_ , std::unique_ptr settings_) + : DatabaseMaterializeMySQL(std::make_shared(database_name_, metadata_path_, context), database_engine_define_->clone(), "MaterializeMySQLSyncThread") + , global_context(context.getGlobalContext()), metadata_path(metadata_path_), mysql_database_name(mysql_database_name_) + , pool(std::move(pool_)), client(std::move(client_)), settings(std::move(settings_)) +{ + +}*/ + +void MaterializeMySQLSyncThread::synchronization() +{ + try + { + if (std::optional metadata = prepareSynchronized()) + { + Stopwatch watch; + Buffers buffers(database_name); + + while (!isCancelled()) + { + /// TODO: add gc task for `sign = -1`(use alter table delete, execute by interval. need final state) + UInt64 max_flush_time = settings->max_flush_data_time; + BinlogEventPtr binlog_event = client.readOneBinlogEvent(std::max(UInt64(1), max_flush_time - watch.elapsedMilliseconds())); + + { + std::unique_lock lock(sync_mutex); + + if (binlog_event) + onEvent(buffers, binlog_event, *metadata); + + if (watch.elapsedMilliseconds() > max_flush_time || buffers.checkThresholds( + settings->max_rows_in_buffer, settings->max_bytes_in_buffer, + settings->max_rows_in_buffers, settings->max_bytes_in_buffers) + ) + { + watch.restart(); + flushBuffersData(buffers, *metadata); + } + } + } + } + } + catch (...) + { + /// TODO: set + getDatabase(database_name).setException(std::current_exception()); + } +} + +void MaterializeMySQLSyncThread::startSynchronization() +{ + if (!background_thread_pool->joinable()) + throw Exception("", ErrorCodes::LOGICAL_ERROR); + + background_thread_pool = std::make_unique([this]() { synchronization(); }); +} + +static inline void cleanOutdatedTables(const String & database_name, const Context & context) +{ + auto ddl_guard = DatabaseCatalog::instance().getDDLGuard(database_name, ""); + const DatabasePtr & clean_database = DatabaseCatalog::instance().getDatabase(database_name); + + for (auto iterator = clean_database->getTablesIterator(context); iterator->isValid(); iterator->next()) + { + String table = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(iterator->name()); + String comment = String("Clean ") + table + " for dump mysql."; + tryToExecuteQuery("DROP TABLE " + table, context, comment); + } +} + +static inline BlockOutputStreamPtr getTableOutput(const String & database_name, const String & table_name, const Context & context) +{ + String with_database_table_name = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name); + BlockIO res = tryToExecuteQuery("INSERT INTO " + with_database_table_name + " VALUES", context, ""); + + if (!res.out) + throw Exception("LOGICAL ERROR:", ErrorCodes::LOGICAL_ERROR); + + return res.out; +} + +static inline void dumpDataForTables( + mysqlxx::Pool::Entry & connection, MaterializeMetadata & master_info, + const String & database_name, const String & mysql_database_name, + const Context & context, const std::function & is_cancelled) +{ + auto iterator = master_info.need_dumping_tables.begin(); + for (; iterator != master_info.need_dumping_tables.end() && !is_cancelled(); ++iterator) + { + const auto & table_name = iterator->first; + MySQLTableStruct table_struct = visitCreateQuery(iterator->second, context, database_name); + String comment = String("Dumping ") + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name); + tryToExecuteQuery(toCreateQuery(table_struct, context), context, comment); + + BlockOutputStreamPtr out = std::make_shared(master_info.version, getTableOutput(database_name, table_name, context)); + MySQLBlockInputStream input( + connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name), + out->getHeader(), DEFAULT_BLOCK_SIZE); + copyData(input, *out, is_cancelled); + } +} + +std::optional MaterializeMySQLSyncThread::prepareSynchronized() +{ + std::unique_lock lock(sync_mutex); + + while (!isCancelled()) + { + try + { + LOG_DEBUG(log, "Checking database status."); + while (!isCancelled() && !DatabaseCatalog::instance().isDatabaseExist(database_name)) + sync_cond.wait_for(lock, std::chrono::seconds(1)); + LOG_DEBUG(log, "Database status is OK."); + + mysqlxx::PoolWithFailover::Entry connection = pool.get(); + MaterializeMetadata metadata(connection, getDatabase(database_name).getMetadataPath() + "/.metadata", mysql_database_name); + + if (!metadata.need_dumping_tables.empty()) + { + metadata.transaction(Position(metadata.binlog_position, metadata.binlog_file), [&]() + { + cleanOutdatedTables(database_name, global_context); + dumpDataForTables(connection, metadata, database_name, mysql_database_name, global_context, [this] { return isCancelled(); }); + }); + } + + client.connect(); + client.startBinlogDump(std::rand(), mysql_database_name, metadata.binlog_file, metadata.binlog_position); + return metadata; + } + catch (mysqlxx::Exception & ) + { + tryLogCurrentException(log); + + /// Avoid busy loop when MySQL is not available. + sleepForMilliseconds(settings->max_wait_time_when_mysql_unavailable); + } + } + + return {}; +} + +void MaterializeMySQLSyncThread::flushBuffersData(Buffers & buffers, MaterializeMetadata & metadata) +{ + metadata.transaction(client.getPosition(), [&]() { buffers.commit(metadata, global_context); }); +} + +static inline void fillSignAndVersionColumnsData(Block & data, Int8 sign_value, UInt64 version_value, size_t fill_size) +{ + MutableColumnPtr sign_mutable_column = IColumn::mutate(std::move(data.getByPosition(data.columns() - 2).column)); + MutableColumnPtr version_mutable_column = IColumn::mutate(std::move(data.getByPosition(data.columns() - 1).column)); + + ColumnInt8::Container & sign_column_data = assert_cast(*sign_mutable_column).getData(); + ColumnUInt64::Container & version_column_data = assert_cast(*version_mutable_column).getData(); + + for (size_t index = 0; index < fill_size; ++index) + { + sign_column_data.emplace_back(sign_value); + version_column_data.emplace_back(version_value); + } +} + +template +static size_t onWriteOrDeleteData(const std::vector & rows_data, Block & buffer, size_t version) +{ + size_t prev_bytes = buffer.bytes(); + for (size_t column = 0; column < buffer.columns() - 2; ++column) + { + MutableColumnPtr col_to = IColumn::mutate(std::move(buffer.getByPosition(column).column)); + + for (size_t index = 0; index < rows_data.size(); ++index) + col_to->insert(DB::get(rows_data[index])[column]); + } + + fillSignAndVersionColumnsData(buffer, sign, version, rows_data.size()); + return buffer.bytes() - prev_bytes; +} + +static inline bool differenceSortingKeys(const Tuple & row_old_data, const Tuple & row_new_data, const std::vector sorting_columns_index) +{ + for (const auto & sorting_column_index : sorting_columns_index) + if (row_old_data[sorting_column_index] != row_new_data[sorting_column_index]) + return true; + + return false; +} + +static inline size_t onUpdateData(const std::vector & rows_data, Block & buffer, size_t version, const std::vector & sorting_columns_index) +{ + if (rows_data.size() % 2 != 0) + throw Exception("LOGICAL ERROR: ", ErrorCodes::LOGICAL_ERROR); + + size_t prev_bytes = buffer.bytes(); + std::vector difference_sorting_keys_mark(rows_data.size() / 2); + + for (size_t index = 0; index < rows_data.size(); index += 2) + difference_sorting_keys_mark.emplace_back(differenceSortingKeys( + DB::get(rows_data[index]), DB::get(rows_data[index + 1]), sorting_columns_index)); + + for (size_t column = 0; column < buffer.columns() - 2; ++column) + { + MutableColumnPtr col_to = IColumn::mutate(std::move(buffer.getByPosition(column).column)); + + for (size_t index = 0; index < rows_data.size(); index += 2) + { + if (likely(!difference_sorting_keys_mark[index / 2])) + col_to->insert(DB::get(rows_data[index + 1])[column]); + else + { + /// If the sorting keys is modified, we should cancel the old data, but this should not happen frequently + col_to->insert(DB::get(rows_data[index])[column]); + col_to->insert(DB::get(rows_data[index + 1])[column]); + } + } + } + + MutableColumnPtr sign_mutable_column = IColumn::mutate(std::move(buffer.getByPosition(buffer.columns() - 2).column)); + MutableColumnPtr version_mutable_column = IColumn::mutate(std::move(buffer.getByPosition(buffer.columns() - 1).column)); + + ColumnInt8::Container & sign_column_data = assert_cast(*sign_mutable_column).getData(); + ColumnUInt64::Container & version_column_data = assert_cast(*version_mutable_column).getData(); + + for (size_t index = 0; index < rows_data.size(); index += 2) + { + if (likely(!difference_sorting_keys_mark[index / 2])) + { + sign_column_data.emplace_back(1); + version_column_data.emplace_back(version); + } + else + { + /// If the sorting keys is modified, we should cancel the old data, but this should not happen frequently + sign_column_data.emplace_back(-1); + sign_column_data.emplace_back(1); + version_column_data.emplace_back(version); + version_column_data.emplace_back(version); + } + } + + return buffer.bytes() - prev_bytes; +} + +void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr & receive_event, MaterializeMetadata & metadata) +{ + if (receive_event->type() == MYSQL_WRITE_ROWS_EVENT) + { + WriteRowsEvent & write_rows_event = static_cast(*receive_event); + Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(write_rows_event.table, global_context); + size_t bytes = onWriteOrDeleteData<1>(write_rows_event.rows, buffer->first, ++metadata.version); + buffers.add(buffer->first.rows(), buffer->first.bytes(), write_rows_event.rows.size(), bytes); + } + else if (receive_event->type() == MYSQL_UPDATE_ROWS_EVENT) + { + UpdateRowsEvent & update_rows_event = static_cast(*receive_event); + Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(update_rows_event.table, global_context); + size_t bytes = onUpdateData(update_rows_event.rows, buffer->first, ++metadata.version, buffer->second); + buffers.add(buffer->first.rows(), buffer->first.bytes(), update_rows_event.rows.size(), bytes); + } + else if (receive_event->type() == MYSQL_DELETE_ROWS_EVENT) + { + DeleteRowsEvent & delete_rows_event = static_cast(*receive_event); + Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(delete_rows_event.table, global_context); + size_t bytes = onWriteOrDeleteData<-1>(delete_rows_event.rows, buffer->first, ++metadata.version); + buffers.add(buffer->first.rows(), buffer->first.bytes(), delete_rows_event.rows.size(), bytes); + } + else if (receive_event->type() == MYSQL_QUERY_EVENT) + { + flushBuffersData(buffers, metadata); + /// TODO: 执行DDL. + /// TODO: 直接使用Interpreter执行即可 + } +} + +void MaterializeMySQLSyncThread::Buffers::add(size_t block_rows, size_t block_bytes, size_t written_rows, size_t written_bytes) +{ + total_blocks_rows += written_rows; + total_blocks_bytes += written_bytes; + max_block_rows = std::max(block_rows, max_block_rows); + max_block_bytes = std::max(block_bytes, max_block_bytes); +} + +bool MaterializeMySQLSyncThread::Buffers::checkThresholds(size_t check_block_rows, size_t check_block_bytes, size_t check_total_rows, size_t check_total_bytes) +{ + return max_block_rows >= check_block_bytes || max_block_bytes >= check_block_bytes || total_blocks_rows >= check_total_rows + || total_blocks_bytes >= check_total_bytes; +} + +void MaterializeMySQLSyncThread::Buffers::commit(MaterializeMetadata & metatdata, const Context & context) +{ + try + { + for (auto & table_name_and_buffer : data) + { + OneBlockInputStream input(table_name_and_buffer.second->first); + BlockOutputStreamPtr out = getTableOutput(database, table_name_and_buffer.first, context); + copyData(input, *out); + } + + data.clear(); + max_block_rows = 0; + max_block_bytes = 0; + total_blocks_rows = 0; + total_blocks_bytes = 0; + } + catch(...) + { + data.clear(); + throw; + } +} + +MaterializeMySQLSyncThread::Buffers::BufferAndSortingColumnsPtr MaterializeMySQLSyncThread::Buffers::getTableDataBuffer( + const String & table_name, const Context & context) +{ + const auto & iterator = data.find(table_name); + if (iterator == data.end()) + { + StoragePtr storage = getDatabase(database).tryGetTable(table_name, context); + + BufferAndSortingColumnsPtr & buffer_and_soring_columns = data.try_emplace( + table_name, std::make_shared(storage->getSampleBlockNonMaterialized(), std::vector{})).first->second; + + if (StorageMergeTree * table_merge_tree = storage->as()) + { + Names required_for_sorting_key = table_merge_tree->getColumnsRequiredForSortingKey(); + + for (const auto & required_name_for_sorting_key : required_for_sorting_key) + buffer_and_soring_columns->second.emplace_back( + buffer_and_soring_columns->first.getPositionByName(required_name_for_sorting_key)); + } + + return buffer_and_soring_columns; + } + + return iterator->second; +} + +} + +#endif diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h new file mode 100644 index 00000000000..94ff2c0b11f --- /dev/null +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -0,0 +1,91 @@ +#pragma once + +#include "config_core.h" + +#if USE_MYSQL + +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include + +namespace DB +{ + +class MaterializeMySQLSyncThread +{ +public: + ~MaterializeMySQLSyncThread(); + + MaterializeMySQLSyncThread( + const Context & context, const String & database_name_, const String & mysql_database_name_ + , mysqlxx::Pool && pool_, MySQLClient && client_, MaterializeMySQLSettings * settings_); + + void startSynchronization(); + +private: + Poco::Logger * log; + const Context & global_context; + + String database_name; + String mysql_database_name; + + mutable mysqlxx::Pool pool; + mutable MySQLClient client; + MaterializeMySQLSettings * settings; + + struct Buffers + { + String database; + + /// thresholds + size_t max_block_rows = 0; + size_t max_block_bytes = 0; + size_t total_blocks_rows = 0; + size_t total_blocks_bytes = 0; + + using BufferAndSortingColumns = std::pair>; + using BufferAndSortingColumnsPtr = std::shared_ptr; + std::unordered_map data; + + Buffers(const String & database_) : database(database_) {} + + void commit(MaterializeMetadata & metatdata, const Context & context); + + void add(size_t block_rows, size_t block_bytes, size_t written_rows, size_t written_bytes); + + bool checkThresholds(size_t check_block_rows, size_t check_block_bytes, size_t check_total_rows, size_t check_total_bytes); + + BufferAndSortingColumnsPtr getTableDataBuffer(const String & table, const Context & context); + }; + + void synchronization(); + + bool isCancelled() { return sync_quit.load(std::memory_order_relaxed); } + + std::optional prepareSynchronized(); + + void flushBuffersData(Buffers & buffers, MaterializeMetadata & metadata); + + void onEvent(Buffers & buffers, const MySQLReplication::BinlogEventPtr & event, MaterializeMetadata & metadata); + + std::mutex sync_mutex; + std::atomic sync_quit{false}; + std::condition_variable sync_cond; + std::unique_ptr background_thread_pool; +}; + +} + +#endif From 1df8bfe4fb92ac1d3638f91d8cb9d2e907645865 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 23 Jun 2020 14:31:22 +0800 Subject: [PATCH 071/374] ISSUES-4006 fix build failure after refactor --- .../MySQL/DatabaseConnectionMySQL.cpp | 10 +++---- .../MySQL/DatabaseMaterializeMySQL.cpp | 11 +++++++- .../MySQL/DatabaseMaterializeMySQL.h | 8 ++++++ .../MySQL/DatabaseMaterializeTablesIterator.h | 1 + .../MySQL/MaterializeMySQLSyncThread.cpp | 27 ++++++++++--------- .../MySQL/MaterializeMySQLSyncThread.h | 3 +-- src/Databases/MySQL/queryConvert.cpp | 3 ++- src/Server/MySQLHandler.cpp | 6 ++--- 8 files changed, 44 insertions(+), 25 deletions(-) diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 7f6fc59c8ed..3b6b0e10b4c 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -89,7 +89,7 @@ bool DatabaseConnectionMySQL::empty() const return true; } -DatabaseTablesIteratorPtr DatabaseConnectionMySQL::getTablesIterator(const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseConnectionMySQL::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) { Tables tables; std::lock_guard lock(mutex); @@ -103,12 +103,12 @@ DatabaseTablesIteratorPtr DatabaseConnectionMySQL::getTablesIterator(const Filte return std::make_unique(tables, database_name); } -bool DatabaseConnectionMySQL::isTableExist(const String & name) const +bool DatabaseConnectionMySQL::isTableExist(const String & name, const Context & context) const { - return bool(tryGetTable(name, global_context)); + return bool(tryGetTable(name, context)); } -StoragePtr DatabaseConnectionMySQL::tryGetTable(const String & mysql_table_name) const +StoragePtr DatabaseConnectionMySQL::tryGetTable(const String & mysql_table_name, const Context &) const { std::lock_guard lock(mutex); @@ -156,7 +156,7 @@ static ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr return create_table_query; } -ASTPtr DatabaseConnectionMySQL::getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const +ASTPtr DatabaseConnectionMySQL::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const { std::lock_guard lock(mutex); diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 32b2e7c3b40..e54b1a3d8b7 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -38,7 +38,16 @@ DatabasePtr DatabaseMaterializeMySQL::getNestedDatabase() const std::unique_lock lock(mutex); if (exception) - std::rethrow_exception(exception); + { + try + { + std::rethrow_exception(exception); + } + catch (Exception & exception) + { + throw Exception(exception); + } + } return nested_database; } diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h index e698bf67689..a30390a4628 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -1,5 +1,9 @@ #pragma once +#include "config_core.h" + +#if USE_MYSQL + #include #include #include @@ -32,6 +36,8 @@ protected: DatabasePtr getNestedDatabase() const; public: + String getEngineName() const override { return "MySQL"; } + ASTPtr getCreateDatabaseQuery() const override; void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override; @@ -78,3 +84,5 @@ public: }; } + +#endif diff --git a/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h b/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h index 805ae3e4c6e..16fd86fc59a 100644 --- a/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h +++ b/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 83f7b45ed28..4ded25a0b7b 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -12,6 +12,7 @@ # include # include # include +# include # include # include # include @@ -33,16 +34,12 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; } -namespace CurrentMetrics -{ - extern const Metric MemoryTracking; - extern const Metric BackgroundMySQLSyncSchedulePoolTask; -} - static BlockIO tryToExecuteQuery(const String & query_to_execute, const Context & context_, const String & comment) { try { + LOG_DEBUG(&Poco::Logger::get("MaterializeMySQLSyncThread"), "Try execute query: " + query_to_execute); + Context context = context_; context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; context.setCurrentQueryId(""); // generate random query_id @@ -92,8 +89,8 @@ MaterializeMySQLSyncThread::~MaterializeMySQLSyncThread() MaterializeMySQLSyncThread::MaterializeMySQLSyncThread( const Context & context, const String & database_name_, const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, MaterializeMySQLSettings * settings_) - : global_context(context), database_name(database_name_), mysql_database_name(mysql_database_name_) - , pool(std::move(pool_)), client(std::move(client_)), settings(settings_) + : log(&Poco::Logger::get("MaterializeMySQLSyncThread")), global_context(context), database_name(database_name_) + , mysql_database_name(mysql_database_name_), pool(std::move(pool_)), client(std::move(client_)), settings(settings_) { /// TODO: 做简单的check, 失败即报错 startSynchronization(); @@ -112,6 +109,8 @@ MaterializeMySQLSyncThread::MaterializeMySQLSyncThread( void MaterializeMySQLSyncThread::synchronization() { + setThreadName("MySQLDBSync"); + try { if (std::optional metadata = prepareSynchronized()) @@ -146,14 +145,15 @@ void MaterializeMySQLSyncThread::synchronization() catch (...) { /// TODO: set + tryLogCurrentException(log); getDatabase(database_name).setException(std::current_exception()); } } void MaterializeMySQLSyncThread::startSynchronization() { - if (!background_thread_pool->joinable()) - throw Exception("", ErrorCodes::LOGICAL_ERROR); +// if (!background_thread_pool->joinable()) +// throw Exception("", ErrorCodes::LOGICAL_ERROR); background_thread_pool = std::make_unique([this]() { synchronization(); }); } @@ -216,6 +216,7 @@ std::optional MaterializeMySQLSyncThread::prepareSynchroniz sync_cond.wait_for(lock, std::chrono::seconds(1)); LOG_DEBUG(log, "Database status is OK."); + mysqlxx::PoolWithFailover::Entry connection = pool.get(); MaterializeMetadata metadata(connection, getDatabase(database_name).getMetadataPath() + "/.metadata", mysql_database_name); @@ -246,7 +247,7 @@ std::optional MaterializeMySQLSyncThread::prepareSynchroniz void MaterializeMySQLSyncThread::flushBuffersData(Buffers & buffers, MaterializeMetadata & metadata) { - metadata.transaction(client.getPosition(), [&]() { buffers.commit(metadata, global_context); }); + metadata.transaction(client.getPosition(), [&]() { buffers.commit(global_context); }); } static inline void fillSignAndVersionColumnsData(Block & data, Int8 sign_value, UInt64 version_value, size_t fill_size) @@ -385,11 +386,11 @@ void MaterializeMySQLSyncThread::Buffers::add(size_t block_rows, size_t block_by bool MaterializeMySQLSyncThread::Buffers::checkThresholds(size_t check_block_rows, size_t check_block_bytes, size_t check_total_rows, size_t check_total_bytes) { - return max_block_rows >= check_block_bytes || max_block_bytes >= check_block_bytes || total_blocks_rows >= check_total_rows + return max_block_rows >= check_block_rows || max_block_bytes >= check_block_bytes || total_blocks_rows >= check_total_rows || total_blocks_bytes >= check_total_bytes; } -void MaterializeMySQLSyncThread::Buffers::commit(MaterializeMetadata & metatdata, const Context & context) +void MaterializeMySQLSyncThread::Buffers::commit(const Context & context) { try { diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index 94ff2c0b11f..68013e08148 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -12,7 +12,6 @@ # include # include # include -# include # include # include # include @@ -61,7 +60,7 @@ private: Buffers(const String & database_) : database(database_) {} - void commit(MaterializeMetadata & metatdata, const Context & context); + void commit(const Context & context); void add(size_t block_rows, size_t block_bytes, size_t written_rows, size_t written_bytes); diff --git a/src/Databases/MySQL/queryConvert.cpp b/src/Databases/MySQL/queryConvert.cpp index 0414c654bd6..cc157f47111 100644 --- a/src/Databases/MySQL/queryConvert.cpp +++ b/src/Databases/MySQL/queryConvert.cpp @@ -138,7 +138,8 @@ String toCreateQuery(const MySQLTableStruct & table_struct, const Context & cont { /// TODO: settings if (table_struct.primary_keys.empty()) - throw Exception("", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("The " + backQuoteIfNeed(table_struct.database_name) + "." + backQuoteIfNeed(table_struct.table_name) + + " cannot be materialized, because there is no primary keys.", ErrorCodes::NOT_IMPLEMENTED); WriteBufferFromOwnString out; String sign = getUniqueColumnName(table_struct.columns_name_and_type, "__sign"); diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 961cf1286cb..34c26fe2bad 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -218,7 +218,7 @@ void MySQLHandler::finishHandshake(MySQLProtocol::HandshakeResponse & packet) copyData(*packet_sender->in, buf_for_handshake_response, packet_size - pos); ReadBufferFromString payload(buf_for_handshake_response.str()); payload.ignore(PACKET_HEADER_SIZE); - packet.readPayload(payload); + packet.readPayloadImpl(payload); packet_sender->sequence_id++; } } @@ -259,7 +259,7 @@ void MySQLHandler::comInitDB(ReadBuffer & payload) void MySQLHandler::comFieldList(ReadBuffer & payload) { ComFieldList packet; - packet.readPayload(payload); + packet.readPayloadImpl(payload); String database = connection_context.getCurrentDatabase(); StoragePtr table_ptr = DatabaseCatalog::instance().getTable({database, packet.table}, connection_context); auto metadata_snapshot = table_ptr->getInMemoryMetadataPtr(); @@ -350,7 +350,7 @@ void MySQLHandlerSSL::finishHandshakeSSL(size_t packet_size, char * buf, size_t SSLRequest ssl_request; ReadBufferFromMemory payload(buf, pos); payload.ignore(PACKET_HEADER_SIZE); - ssl_request.readPayload(payload); + ssl_request.readPayloadImpl(payload); connection_context.mysql.client_capabilities = ssl_request.capability_flags; connection_context.mysql.max_packet_size = ssl_request.max_packet_size ? ssl_request.max_packet_size : MAX_PACKET_LENGTH; secure_connection = true; From 9324c0ee497e311b886277054d43b4c184d6cb4c Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 24 Jun 2020 01:45:55 +0800 Subject: [PATCH 072/374] ISSUES-4006 fix crash when dump data --- .../MySQL/MaterializeMySQLSyncThread.cpp | 23 +++++++++++++++---- src/Storages/StorageMaterializeMySQL.cpp | 19 +++++++++++++++ 2 files changed, 38 insertions(+), 4 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 4ded25a0b7b..16187585ce4 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -40,7 +40,8 @@ static BlockIO tryToExecuteQuery(const String & query_to_execute, const Context { LOG_DEBUG(&Poco::Logger::get("MaterializeMySQLSyncThread"), "Try execute query: " + query_to_execute); - Context context = context_; + Context context(context_); + CurrentThread::QueryScope query_scope(context); context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; context.setCurrentQueryId(""); // generate random query_id return executeQuery("/*" + comment + "*/ " + query_to_execute, context, true); @@ -51,7 +52,7 @@ static BlockIO tryToExecuteQuery(const String & query_to_execute, const Context throw; } -// LOG_DEBUG(&Logger::get("MaterializeMySQLSyncThread"), "Executed query: " + query_to_execute); + LOG_DEBUG(&Poco::Logger::get("MaterializeMySQLSyncThread"), "Executed query: " + query_to_execute); } static inline DatabaseMaterializeMySQL & getDatabase(const String & database_name) @@ -89,7 +90,7 @@ MaterializeMySQLSyncThread::~MaterializeMySQLSyncThread() MaterializeMySQLSyncThread::MaterializeMySQLSyncThread( const Context & context, const String & database_name_, const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, MaterializeMySQLSettings * settings_) - : log(&Poco::Logger::get("MaterializeMySQLSyncThread")), global_context(context), database_name(database_name_) + : log(&Poco::Logger::get("MaterializeMySQLSyncThread")), global_context(context.getGlobalContext()), database_name(database_name_) , mysql_database_name(mysql_database_name_), pool(std::move(pool_)), client(std::move(client_)), settings(settings_) { /// TODO: 做简单的check, 失败即报错 @@ -128,7 +129,10 @@ void MaterializeMySQLSyncThread::synchronization() std::unique_lock lock(sync_mutex); if (binlog_event) + { + binlog_event->dump(); onEvent(buffers, binlog_event, *metadata); + } if (watch.elapsedMilliseconds() > max_flush_time || buffers.checkThresholds( settings->max_rows_in_buffer, settings->max_bytes_in_buffer, @@ -136,7 +140,9 @@ void MaterializeMySQLSyncThread::synchronization() ) { watch.restart(); - flushBuffersData(buffers, *metadata); + + if (!buffers.data.empty()) + flushBuffersData(buffers, *metadata); } } } @@ -263,6 +269,9 @@ static inline void fillSignAndVersionColumnsData(Block & data, Int8 sign_value, sign_column_data.emplace_back(sign_value); version_column_data.emplace_back(version_value); } + + data.getByPosition(data.columns() - 2).column = std::move(sign_mutable_column); + data.getByPosition(data.columns() - 1).column = std::move(version_mutable_column); } template @@ -275,6 +284,8 @@ static size_t onWriteOrDeleteData(const std::vector & rows_data, Block & for (size_t index = 0; index < rows_data.size(); ++index) col_to->insert(DB::get(rows_data[index])[column]); + + buffer.getByPosition(column).column = std::move(col_to); } fillSignAndVersionColumnsData(buffer, sign, version, rows_data.size()); @@ -317,6 +328,8 @@ static inline size_t onUpdateData(const std::vector & rows_data, Block & col_to->insert(DB::get(rows_data[index + 1])[column]); } } + + buffer.getByPosition(column).column = std::move(col_to); } MutableColumnPtr sign_mutable_column = IColumn::mutate(std::move(buffer.getByPosition(buffer.columns() - 2).column)); @@ -342,6 +355,8 @@ static inline size_t onUpdateData(const std::vector & rows_data, Block & } } + buffer.getByPosition(buffer.columns() - 2).column = std::move(sign_mutable_column); + buffer.getByPosition(buffer.columns() - 1).column = std::move(version_mutable_column); return buffer.bytes() - prev_bytes; } diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index 6bcdc9bd3b8..aba9ea061e0 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -1,9 +1,14 @@ #include +#include #include #include +#include +#include + #include +#include namespace DB { @@ -51,6 +56,20 @@ Pipes StorageMaterializeMySQL::read( require_columns_name.emplace_back(sign_column.name); return nested_storage->read(require_columns_name, query_info, context, processed_stage, max_block_size, num_streams); + + /*for (auto & pipe : pipes) + { + std::cout << "Pipe Header Structure:" << pipe.getHeader().dumpStructure() << "\n"; + ASTPtr expr = makeASTFunction( + "equals", std::make_shared(sign_column.name), std::make_shared(Field(Int8(1)))); + auto syntax = SyntaxAnalyzer(context).analyze(expr, pipe.getHeader().getNamesAndTypesList()); + ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expr, syntax, context).getActions(true); + + pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), expression_actions, expr->getColumnName(), false)); + /// TODO: maybe need remove sign columns + }*/ + +// return pipes; } } From a9e1794bd3ec1f1eb13d7e4d1c199256ac522b9c Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 24 Jun 2020 13:28:27 +0800 Subject: [PATCH 073/374] ISSUES-4006 filter sign = -1 rows --- .../MySQL/MaterializeMySQLSyncThread.cpp | 5 +- src/Storages/StorageMaterializeMySQL.cpp | 63 ++++++++++++++----- src/Storages/StorageMaterializeMySQL.h | 2 + 3 files changed, 52 insertions(+), 18 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 16187585ce4..caaec6defcf 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -310,8 +310,8 @@ static inline size_t onUpdateData(const std::vector & rows_data, Block & std::vector difference_sorting_keys_mark(rows_data.size() / 2); for (size_t index = 0; index < rows_data.size(); index += 2) - difference_sorting_keys_mark.emplace_back(differenceSortingKeys( - DB::get(rows_data[index]), DB::get(rows_data[index + 1]), sorting_columns_index)); + difference_sorting_keys_mark[index / 2] = differenceSortingKeys( + DB::get(rows_data[index]), DB::get(rows_data[index + 1]), sorting_columns_index); for (size_t column = 0; column < buffer.columns() - 2; ++column) { @@ -447,6 +447,7 @@ MaterializeMySQLSyncThread::Buffers::BufferAndSortingColumnsPtr MaterializeMySQL for (const auto & required_name_for_sorting_key : required_for_sorting_key) buffer_and_soring_columns->second.emplace_back( buffer_and_soring_columns->first.getPositionByName(required_name_for_sorting_key)); + } return buffer_and_soring_columns; diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index aba9ea061e0..03a1d511d26 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -35,7 +35,13 @@ Pipes StorageMaterializeMySQL::read( size_t max_block_size, unsigned int num_streams) { - if (ASTSelectQuery * select_query = query_info.query->as()) + NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); + + Block nested_header = nested_storage->getSampleBlockNonMaterialized(); + ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); + ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); + + if (ASTSelectQuery * select_query = query_info.query->as(); select_query && !column_names_set.count(version_column.name)) { auto & tables_in_select_query = select_query->tables()->as(); @@ -48,28 +54,53 @@ Pipes StorageMaterializeMySQL::read( } } + String filter_column_name; Names require_columns_name = column_names; - Block header = nested_storage->getSampleBlockNonMaterialized(); - ColumnWithTypeAndName & sign_column = header.getByPosition(header.columns() - 2); - - if (require_columns_name.end() == std::find(require_columns_name.begin(), require_columns_name.end(), sign_column.name)) + ASTPtr expressions = std::make_shared(); + if (column_names_set.empty() || !column_names_set.count(sign_column.name)) + { require_columns_name.emplace_back(sign_column.name); - return nested_storage->read(require_columns_name, query_info, context, processed_stage, max_block_size, num_streams); + const auto & sign_column_name = std::make_shared(sign_column.name); + const auto & fetch_sign_value = std::make_shared(Field(Int8(1))); - /*for (auto & pipe : pipes) + expressions->children.emplace_back(makeASTFunction("equals", sign_column_name, fetch_sign_value)); + filter_column_name = expressions->children.back()->getColumnName(); + + for (const auto & column_name : column_names) + expressions->children.emplace_back(std::make_shared(column_name)); + } + + Pipes pipes = nested_storage->read(require_columns_name, query_info, context, processed_stage, max_block_size, num_streams); + + if (!expressions->children.empty() && !pipes.empty()) { - std::cout << "Pipe Header Structure:" << pipe.getHeader().dumpStructure() << "\n"; - ASTPtr expr = makeASTFunction( - "equals", std::make_shared(sign_column.name), std::make_shared(Field(Int8(1)))); - auto syntax = SyntaxAnalyzer(context).analyze(expr, pipe.getHeader().getNamesAndTypesList()); - ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expr, syntax, context).getActions(true); + Block pipe_header = pipes.front().getHeader(); + SyntaxAnalyzerResultPtr syntax = SyntaxAnalyzer(context).analyze(expressions, pipe_header.getNamesAndTypesList()); + ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true); - pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), expression_actions, expr->getColumnName(), false)); - /// TODO: maybe need remove sign columns - }*/ + for (auto & pipe : pipes) + { + assertBlocksHaveEqualStructure(pipe_header, pipe.getHeader(), "StorageMaterializeMySQL"); + pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), expression_actions, filter_column_name, false)); + } + } -// return pipes; + return pipes; +} + +NamesAndTypesList StorageMaterializeMySQL::getVirtuals() const +{ + NamesAndTypesList virtuals; + Block nested_header = nested_storage->getSampleBlockNonMaterialized(); + ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); + ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); + virtuals.emplace_back(NameAndTypePair(sign_column.name, sign_column.type)); + virtuals.emplace_back(NameAndTypePair(version_column.name, version_column.type)); + + auto nested_virtuals = nested_storage->getVirtuals(); + virtuals.insert(virtuals.end(), nested_virtuals.begin(), nested_virtuals.end()); + return virtuals; } } diff --git a/src/Storages/StorageMaterializeMySQL.h b/src/Storages/StorageMaterializeMySQL.h index d88e72ed737..a74f2208a9a 100644 --- a/src/Storages/StorageMaterializeMySQL.h +++ b/src/Storages/StorageMaterializeMySQL.h @@ -21,6 +21,8 @@ public: const Names & column_names, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; + NamesAndTypesList getVirtuals() const override; + private: StoragePtr nested_storage; }; From 101a13b5c37c3d4835187bbedbecaeb9f5190b0e Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 24 Jun 2020 14:04:21 +0800 Subject: [PATCH 074/374] ISSUES-4006 remove unused code --- .../MySQL/MaterializeMySQLSyncThread.cpp | 20 +++---------------- 1 file changed, 3 insertions(+), 17 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index caaec6defcf..0ecf3c49d10 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -62,7 +62,7 @@ static inline DatabaseMaterializeMySQL & getDatabase(const String & database_nam if (DatabaseMaterializeMySQL * database_materialize = typeid_cast(database.get())) return *database_materialize; - throw Exception("", ErrorCodes::LOGICAL_ERROR); + throw Exception("LOGICAL_ERROR: cannot cast to DatabaseMaterializeMySQL, it is a bug.", ErrorCodes::LOGICAL_ERROR); } MaterializeMySQLSyncThread::~MaterializeMySQLSyncThread() @@ -77,8 +77,7 @@ MaterializeMySQLSyncThread::~MaterializeMySQLSyncThread() } sync_cond.notify_one(); -// sync_task_handler->deactivate(); -// flush_task_handler->deactivate(); + /// TODO: join thread } } catch (...) @@ -97,17 +96,6 @@ MaterializeMySQLSyncThread::MaterializeMySQLSyncThread( startSynchronization(); } -/*MaterializeMySQLSyncThread::MaterializeMySQLSyncThread( - const Context & context, const String & database_name_, const String & metadata_path_ - , const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_ - , MySQLClient && client_ , std::unique_ptr settings_) - : DatabaseMaterializeMySQL(std::make_shared(database_name_, metadata_path_, context), database_engine_define_->clone(), "MaterializeMySQLSyncThread") - , global_context(context.getGlobalContext()), metadata_path(metadata_path_), mysql_database_name(mysql_database_name_) - , pool(std::move(pool_)), client(std::move(client_)), settings(std::move(settings_)) -{ - -}*/ - void MaterializeMySQLSyncThread::synchronization() { setThreadName("MySQLDBSync"); @@ -158,9 +146,7 @@ void MaterializeMySQLSyncThread::synchronization() void MaterializeMySQLSyncThread::startSynchronization() { -// if (!background_thread_pool->joinable()) -// throw Exception("", ErrorCodes::LOGICAL_ERROR); - + /// TODO: reset exception. background_thread_pool = std::make_unique([this]() { synchronization(); }); } From 3afffaf30338e458e80b3cce569a1cd2730a3ed9 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 25 Jun 2020 23:39:33 +0800 Subject: [PATCH 075/374] ISSUES-4006 rename sign and version column --- src/Databases/MySQL/queryConvert.cpp | 4 +-- src/Parsers/MySQL/tryParseMySQLQuery.cpp | 37 ++++++++++++++++++++++++ src/Parsers/MySQL/tryParseMySQLQuery.h | 24 +++++++++++++++ 3 files changed, 63 insertions(+), 2 deletions(-) create mode 100644 src/Parsers/MySQL/tryParseMySQLQuery.cpp create mode 100644 src/Parsers/MySQL/tryParseMySQLQuery.h diff --git a/src/Databases/MySQL/queryConvert.cpp b/src/Databases/MySQL/queryConvert.cpp index cc157f47111..7e062413523 100644 --- a/src/Databases/MySQL/queryConvert.cpp +++ b/src/Databases/MySQL/queryConvert.cpp @@ -142,8 +142,8 @@ String toCreateQuery(const MySQLTableStruct & table_struct, const Context & cont + " cannot be materialized, because there is no primary keys.", ErrorCodes::NOT_IMPLEMENTED); WriteBufferFromOwnString out; - String sign = getUniqueColumnName(table_struct.columns_name_and_type, "__sign"); - String version = getUniqueColumnName(table_struct.columns_name_and_type, "__version"); + String sign = getUniqueColumnName(table_struct.columns_name_and_type, "_sign"); + String version = getUniqueColumnName(table_struct.columns_name_and_type, "_version"); out << "CREATE TABLE " << (table_struct.if_not_exists ? "IF NOT EXISTS" : "") << backQuoteIfNeed(table_struct.database_name) + "." << backQuoteIfNeed(table_struct.table_name) << "(" << queryToString(InterpreterCreateQuery::formatColumns(table_struct.columns_name_and_type)) diff --git a/src/Parsers/MySQL/tryParseMySQLQuery.cpp b/src/Parsers/MySQL/tryParseMySQLQuery.cpp new file mode 100644 index 00000000000..27aeefdbb1e --- /dev/null +++ b/src/Parsers/MySQL/tryParseMySQLQuery.cpp @@ -0,0 +1,37 @@ +#include + +#include +#include +#include + +#include + +namespace DB +{ + +namespace MySQLParser +{ + +bool ParserQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserDropQuery p_drop_query; + ParserRenameQuery p_rename_query; + ParserCreateQuery p_create_query; + /// TODO: alter table + + return p_create_query.parse(pos, node, expected) || p_drop_query.parse(pos, node, expected) + || p_rename_query.parse(pos, node, expected); +} + +} + +ASTPtr tryParseMySQLQuery(const std::string & query, size_t max_query_size, size_t max_parser_depth) +{ + std::string error_message; + const char * pos = query.data(); + MySQLParser::ParserQuery p_query; + return tryParseQuery(p_query, pos, query.data() + query.size(), error_message, false, "", false, max_query_size, max_parser_depth); +} + +} + diff --git a/src/Parsers/MySQL/tryParseMySQLQuery.h b/src/Parsers/MySQL/tryParseMySQLQuery.h new file mode 100644 index 00000000000..217a30553ff --- /dev/null +++ b/src/Parsers/MySQL/tryParseMySQLQuery.h @@ -0,0 +1,24 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ParserQuery : public IParserBase +{ +protected: + const char * getName() const override { return "MySQL Query"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} + +ASTPtr tryParseMySQLQuery(const std::string & query, size_t max_query_size, size_t max_parser_depth); + +} From 5fb92359d23d146f47e30ba7a0cb961c3ada9994 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 29 Jun 2020 12:34:49 +0800 Subject: [PATCH 076/374] ISSUES-4006 some refactor for query convert --- .../MySQL/DatabaseMaterializeMySQL.cpp | 19 +++--- .../MySQL/MaterializeMySQLSyncThread.cpp | 8 ++- .../MySQL/MaterializeMySQLSyncThread.h | 2 + src/Interpreters/MySQL/CreateQueryVisitor.cpp | 60 +------------------ .../MySQL/InterpreterMySQLCreateQuery.cpp | 5 ++ .../MySQL/InterpreterMySQLCreateQuery.h | 13 ++++ ...rseMySQLQuery.cpp => ParserMySQLQuery.cpp} | 17 ++---- ...ryParseMySQLQuery.h => ParserMySQLQuery.h} | 4 +- src/Parsers/ParserQuery.cpp | 5 +- 9 files changed, 49 insertions(+), 84 deletions(-) create mode 100644 src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp create mode 100644 src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h rename src/Parsers/MySQL/{tryParseMySQLQuery.cpp => ParserMySQLQuery.cpp} (50%) rename src/Parsers/MySQL/{tryParseMySQLQuery.h => ParserMySQLQuery.h} (66%) diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index e54b1a3d8b7..502fcc4d62b 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -10,8 +11,6 @@ namespace DB { -static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; - namespace ErrorCodes { extern const int NOT_IMPLEMENTED; @@ -121,7 +120,7 @@ time_t DatabaseMaterializeMySQL::getObjectMetadataModificationTime(const String void DatabaseMaterializeMySQL::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) { - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) throw Exception("MySQL database in locality_data mode does not support create table.", ErrorCodes::NOT_IMPLEMENTED); getNestedDatabase()->createTable(context, name, table, query); @@ -129,7 +128,7 @@ void DatabaseMaterializeMySQL::createTable(const Context & context, const String void DatabaseMaterializeMySQL::dropTable(const Context & context, const String & name, bool no_delay) { - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) throw Exception("MySQL database in locality_data mode does not support drop table.", ErrorCodes::NOT_IMPLEMENTED); getNestedDatabase()->dropTable(context, name, no_delay); @@ -137,7 +136,7 @@ void DatabaseMaterializeMySQL::dropTable(const Context & context, const String & void DatabaseMaterializeMySQL::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) { - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) throw Exception("MySQL database in locality_data mode does not support attach table.", ErrorCodes::NOT_IMPLEMENTED); getNestedDatabase()->attachTable(name, table, relative_table_path); @@ -145,7 +144,7 @@ void DatabaseMaterializeMySQL::attachTable(const String & name, const StoragePtr StoragePtr DatabaseMaterializeMySQL::detachTable(const String & name) { - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) throw Exception("MySQL database in locality_data mode does not support detach table.", ErrorCodes::NOT_IMPLEMENTED); return getNestedDatabase()->detachTable(name); @@ -153,7 +152,7 @@ StoragePtr DatabaseMaterializeMySQL::detachTable(const String & name) void DatabaseMaterializeMySQL::renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange) { - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) throw Exception("MySQL database in locality_data mode does not support rename table.", ErrorCodes::NOT_IMPLEMENTED); getNestedDatabase()->renameTable(context, name, to_database, to_name, exchange); @@ -161,7 +160,7 @@ void DatabaseMaterializeMySQL::renameTable(const Context & context, const String void DatabaseMaterializeMySQL::alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) { - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) throw Exception("MySQL database in locality_data mode does not support alter table.", ErrorCodes::NOT_IMPLEMENTED); getNestedDatabase()->alterTable(context, table_id, metadata); @@ -184,7 +183,7 @@ bool DatabaseMaterializeMySQL::isTableExist(const String & name, const Context & StoragePtr DatabaseMaterializeMySQL::tryGetTable(const String & name, const Context & context) const { - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) return std::make_shared(getNestedDatabase()->tryGetTable(name, context)); return getNestedDatabase()->tryGetTable(name, context); @@ -192,7 +191,7 @@ StoragePtr DatabaseMaterializeMySQL::tryGetTable(const String & name, const Cont DatabaseTablesIteratorPtr DatabaseMaterializeMySQL::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) { - if (getThreadName() != MYSQL_BACKGROUND_THREAD_NAME) + if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) { DatabaseTablesIteratorPtr iterator = getNestedDatabase()->getTablesIterator(context, filter_by_table_name); return std::make_unique(std::move(iterator)); diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 0ecf3c49d10..39660cc0a34 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -34,6 +34,8 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; } +static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; + static BlockIO tryToExecuteQuery(const String & query_to_execute, const Context & context_, const String & comment) { try @@ -98,7 +100,7 @@ MaterializeMySQLSyncThread::MaterializeMySQLSyncThread( void MaterializeMySQLSyncThread::synchronization() { - setThreadName("MySQLDBSync"); + setThreadName(MYSQL_BACKGROUND_THREAD_NAME); try { @@ -376,6 +378,10 @@ void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr /// TODO: 直接使用Interpreter执行即可 } } +bool MaterializeMySQLSyncThread::isMySQLSyncThread() +{ + return getThreadName() == MYSQL_BACKGROUND_THREAD_NAME; +} void MaterializeMySQLSyncThread::Buffers::add(size_t block_rows, size_t block_bytes, size_t written_rows, size_t written_bytes) { diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index 68013e08148..38b59fbb736 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -33,6 +33,8 @@ public: void startSynchronization(); + static bool isMySQLSyncThread(); + private: Poco::Logger * log; const Context & global_context; diff --git a/src/Interpreters/MySQL/CreateQueryVisitor.cpp b/src/Interpreters/MySQL/CreateQueryVisitor.cpp index c41b51d54dc..5090e7eb40f 100644 --- a/src/Interpreters/MySQL/CreateQueryVisitor.cpp +++ b/src/Interpreters/MySQL/CreateQueryVisitor.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include @@ -28,55 +27,6 @@ namespace ErrorCodes namespace MySQLVisitor { -static String convertDataType(const String & type_name, const ASTPtr & arguments, bool is_unsigned) -{ - if (type_name == "TINYINT") - return is_unsigned ? "UInt8" : "Int8"; - else if (type_name == "BOOL" || type_name == "BOOLEAN") - return "UInt8"; - else if (type_name == "SMALLINT") - return is_unsigned ? "UInt16" : "Int16"; - else if (type_name == "INT" || type_name == "MEDIUMINT" || type_name == "INTEGER") - return is_unsigned ? "UInt32" : "Int32"; - else if (type_name == "BIGINT") - return is_unsigned ? "UInt64" : "Int64"; - else if (type_name == "FLOAT") - return "Float32"; - else if (type_name == "DOUBLE" || type_name == "PRECISION" || type_name == "REAL") - return "Float64"; - else if (type_name == "DECIMAL" || type_name == "DEC" || type_name == "NUMERIC" || type_name == "FIXED") - { - if (!arguments) - return "Decimal(10, 0)"; - else if (arguments->children.size() == 1) - return "Decimal(" + queryToString(arguments) + ", 0)"; - else if (arguments->children.size() == 2) - return "Decimal(" + queryToString(arguments) + ")"; - else - throw Exception("Decimal data type family must have exactly two arguments: precision and scale", ErrorCodes::UNKNOWN_TYPE); - } - - - if (type_name == "DATE") - return "Date"; - else if (type_name == "DATETIME" || type_name == "TIMESTAMP") - return "DateTime"; - else if (type_name == "TIME") - return "DateTime64"; - else if (type_name == "YEAR") - return "Int16"; - - if (type_name == "BINARY") - return arguments ? "FixedString(" + queryToString(arguments) + ")" : "FixedString(1)"; - - return "String"; -} - -static String convertDataType(const String & type_name, const ASTPtr & arguments, bool is_unsigned, bool is_nullable) -{ - return (is_nullable ? "Nullable(" : "") + convertDataType(type_name, arguments, is_unsigned) + (is_nullable ? ")" : ""); -} - void CreateQueryMatcher::visit(ASTPtr & ast, Data & data) { if (auto * t = ast->as()) @@ -143,14 +93,8 @@ void CreateQueryMatcher::visit(const MySQLParser::ASTDeclareColumn & declare_col } } - if (ASTFunction * function = declare_column.data_type->as()) - data.columns_name_and_type.emplace_back(declare_column.name, - DataTypeFactory::instance().get(convertDataType(Poco::toUpper(function->name), function->arguments, is_unsigned, is_nullable))); - else if (ASTIdentifier * identifier = declare_column.data_type->as()) - data.columns_name_and_type.emplace_back(declare_column.name, - DataTypeFactory::instance().get(convertDataType(Poco::toUpper(identifier->name), ASTPtr{}, is_unsigned, is_nullable))); - else - throw Exception("Unsupported MySQL data type " + queryToString(declare_column.data_type) + ".", ErrorCodes::NOT_IMPLEMENTED); + data.columns_name_and_type.emplace_back(declare_column.name, DataTypeFactory::instance().get( + (is_nullable ? "Nullable(" : "") + queryToString(declare_column.data_type) + (is_nullable ? ")" : ""))); } void CreateQueryMatcher::visit(const MySQLParser::ASTDeclarePartitionOptions & declare_partition_options, const ASTPtr &, Data & data) diff --git a/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp b/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp new file mode 100644 index 00000000000..160b32b743b --- /dev/null +++ b/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp @@ -0,0 +1,5 @@ +// +// Created by coswde on 2020/6/29. +// + +#include "InterpreterMySQLCreateQuery.h" diff --git a/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h b/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h new file mode 100644 index 00000000000..76bfc2fa2a7 --- /dev/null +++ b/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h @@ -0,0 +1,13 @@ +#pragma once + +namespace DB +{ + +namespace MySQLInterpreter +{ + + + +} + +} diff --git a/src/Parsers/MySQL/tryParseMySQLQuery.cpp b/src/Parsers/MySQL/ParserMySQLQuery.cpp similarity index 50% rename from src/Parsers/MySQL/tryParseMySQLQuery.cpp rename to src/Parsers/MySQL/ParserMySQLQuery.cpp index 27aeefdbb1e..26bc4effbab 100644 --- a/src/Parsers/MySQL/tryParseMySQLQuery.cpp +++ b/src/Parsers/MySQL/ParserMySQLQuery.cpp @@ -1,10 +1,11 @@ -#include +#include #include #include #include #include +#include namespace DB { @@ -12,8 +13,11 @@ namespace DB namespace MySQLParser { -bool ParserQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +bool ParserMySQLQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { + if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) + return false; + ParserDropQuery p_drop_query; ParserRenameQuery p_rename_query; ParserCreateQuery p_create_query; @@ -25,13 +29,4 @@ bool ParserQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expect } -ASTPtr tryParseMySQLQuery(const std::string & query, size_t max_query_size, size_t max_parser_depth) -{ - std::string error_message; - const char * pos = query.data(); - MySQLParser::ParserQuery p_query; - return tryParseQuery(p_query, pos, query.data() + query.size(), error_message, false, "", false, max_query_size, max_parser_depth); } - -} - diff --git a/src/Parsers/MySQL/tryParseMySQLQuery.h b/src/Parsers/MySQL/ParserMySQLQuery.h similarity index 66% rename from src/Parsers/MySQL/tryParseMySQLQuery.h rename to src/Parsers/MySQL/ParserMySQLQuery.h index 217a30553ff..aa52cd88a6e 100644 --- a/src/Parsers/MySQL/tryParseMySQLQuery.h +++ b/src/Parsers/MySQL/ParserMySQLQuery.h @@ -9,7 +9,7 @@ namespace DB namespace MySQLParser { -class ParserQuery : public IParserBase +class ParserMySQLQuery : public IParserBase { protected: const char * getName() const override { return "MySQL Query"; } @@ -19,6 +19,4 @@ protected: } -ASTPtr tryParseMySQLQuery(const std::string & query, size_t max_query_size, size_t max_parser_depth); - } diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 144c309927b..49897b9865d 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -17,6 +17,7 @@ #include #include #include +#include namespace DB @@ -38,6 +39,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserDropAccessEntityQuery drop_access_entity_p; ParserGrantQuery grant_p; ParserSetRoleQuery set_role_p; + MySQLParser::ParserMySQLQuery mysql_query_p; bool res = query_with_output_p.parse(pos, node, expected) || insert_p.parse(pos, node, expected) @@ -51,7 +53,8 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_row_policy_p.parse(pos, node, expected) || create_settings_profile_p.parse(pos, node, expected) || drop_access_entity_p.parse(pos, node, expected) - || grant_p.parse(pos, node, expected); + || grant_p.parse(pos, node, expected) + || mysql_query_p.parse(pos, node, expected); return res; } From 54c3db0567d27b9d5ebf1a8d3f21154fd0bfacc3 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 1 Jul 2020 13:00:50 +0800 Subject: [PATCH 077/374] ISSUES-4006 some refactor for query convert --- src/Databases/MySQL/MaterializeMetadata.cpp | 3 +- src/Databases/MySQL/MaterializeMetadata.h | 2 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 72 +++-- src/Databases/MySQL/queryConvert.cpp | 156 ---------- src/Databases/MySQL/queryConvert.h | 19 -- src/Interpreters/InterpreterFactory.cpp | 11 + src/Interpreters/MySQL/CreateQueryVisitor.cpp | 6 +- .../MySQL/InterpreterMySQLCreateQuery.cpp | 287 +++++++++++++++++- .../MySQL/InterpreterMySQLCreateQuery.h | 17 ++ src/Parsers/MySQL/ASTDeclareColumn.cpp | 21 +- src/Parsers/MySQL/ASTDeclareIndex.cpp | 4 +- src/Parsers/MySQL/ParserMySQLQuery.cpp | 3 +- src/Storages/StorageMaterializeMySQL.cpp | 18 +- src/Storages/StorageMaterializeMySQL.h | 2 +- 14 files changed, 387 insertions(+), 234 deletions(-) delete mode 100644 src/Databases/MySQL/queryConvert.cpp delete mode 100644 src/Databases/MySQL/queryConvert.h diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 768ed4fc57b..73f7b1c3870 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -142,7 +142,7 @@ void MaterializeMetadata::transaction(const MySQLReplication::Position & positio commitMetadata(fun, persistent_tmp_path, persistent_path); } -MaterializeMetadata::MaterializeMetadata(mysqlxx::PoolWithFailover::Entry & connection, const String & path_, const String & database) +MaterializeMetadata::MaterializeMetadata(mysqlxx::PoolWithFailover::Entry & connection, const String & path_, const String & database, bool & opened_transaction) : persistent_path(path_) { if (Poco::File(persistent_path).exists()) @@ -174,6 +174,7 @@ MaterializeMetadata::MaterializeMetadata(mysqlxx::PoolWithFailover::Entry & conn connection->query("SET SESSION TRANSACTION ISOLATION LEVEL REPEATABLE READ;").execute(); connection->query("START TRANSACTION /*!40100 WITH CONSISTENT SNAPSHOT */;").execute(); + opened_transaction = true; need_dumping_tables = fetchTablesCreateQuery(connection, database, fetchTablesInDB(connection, database)); connection->query("UNLOCK TABLES;").execute(); } diff --git a/src/Databases/MySQL/MaterializeMetadata.h b/src/Databases/MySQL/MaterializeMetadata.h index ad2f77a2f42..2020674c86c 100644 --- a/src/Databases/MySQL/MaterializeMetadata.h +++ b/src/Databases/MySQL/MaterializeMetadata.h @@ -27,7 +27,7 @@ struct MaterializeMetadata void transaction(const MySQLReplication::Position & position, const std::function & fun); - MaterializeMetadata(mysqlxx::PoolWithFailover::Entry & connection, const String & path, const String & database); + MaterializeMetadata(mysqlxx::PoolWithFailover::Entry & connection, const String & path, const String & database, bool & opened_transaction); }; } diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 39660cc0a34..69576df42ca 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -13,7 +13,6 @@ # include # include # include -# include # include # include # include @@ -32,29 +31,33 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_QUERY; + extern const int SYNTAX_ERROR; } static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; -static BlockIO tryToExecuteQuery(const String & query_to_execute, const Context & context_, const String & comment) +static BlockIO tryToExecuteQuery(const String & query_to_execute, const Context & context_, const String & database, const String & comment) { try { - LOG_DEBUG(&Poco::Logger::get("MaterializeMySQLSyncThread"), "Try execute query: " + query_to_execute); + LOG_DEBUG(&Poco::Logger::get("MaterializeMySQLSyncThread(" + database + ")"), "Try execute query: " + query_to_execute); Context context(context_); CurrentThread::QueryScope query_scope(context); + context.setCurrentDatabase(database); context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; context.setCurrentQueryId(""); // generate random query_id return executeQuery("/*" + comment + "*/ " + query_to_execute, context, true); } catch (...) { - tryLogCurrentException("MaterializeMySQLSyncThread", "Query " + query_to_execute + " wasn't finished successfully"); + tryLogCurrentException( + &Poco::Logger::get("MaterializeMySQLSyncThread(" + database + ")"), + "Query " + query_to_execute + " wasn't finished successfully"); throw; } - LOG_DEBUG(&Poco::Logger::get("MaterializeMySQLSyncThread"), "Executed query: " + query_to_execute); + LOG_DEBUG(&Poco::Logger::get("MaterializeMySQLSyncThread(" + database + ")"), "Executed query: " + query_to_execute); } static inline DatabaseMaterializeMySQL & getDatabase(const String & database_name) @@ -159,16 +162,15 @@ static inline void cleanOutdatedTables(const String & database_name, const Conte for (auto iterator = clean_database->getTablesIterator(context); iterator->isValid(); iterator->next()) { - String table = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(iterator->name()); - String comment = String("Clean ") + table + " for dump mysql."; - tryToExecuteQuery("DROP TABLE " + table, context, comment); + String table_name = backQuoteIfNeed(iterator->name()); + String comment = String("Clean ") + table_name + " for dump mysql."; + tryToExecuteQuery("DROP TABLE " + table_name, context, backQuoteIfNeed(database_name), comment); } } static inline BlockOutputStreamPtr getTableOutput(const String & database_name, const String & table_name, const Context & context) { - String with_database_table_name = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name); - BlockIO res = tryToExecuteQuery("INSERT INTO " + with_database_table_name + " VALUES", context, ""); + BlockIO res = tryToExecuteQuery("INSERT INTO " + backQuoteIfNeed(table_name) + " VALUES", context, database_name, ""); if (!res.out) throw Exception("LOGICAL ERROR:", ErrorCodes::LOGICAL_ERROR); @@ -185,9 +187,8 @@ static inline void dumpDataForTables( for (; iterator != master_info.need_dumping_tables.end() && !is_cancelled(); ++iterator) { const auto & table_name = iterator->first; - MySQLTableStruct table_struct = visitCreateQuery(iterator->second, context, database_name); String comment = String("Dumping ") + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name); - tryToExecuteQuery(toCreateQuery(table_struct, context), context, comment); + tryToExecuteQuery(iterator->second, context, database_name, comment); /// create table. BlockOutputStreamPtr out = std::make_shared(master_info.version, getTableOutput(database_name, table_name, context)); MySQLBlockInputStream input( @@ -201,6 +202,9 @@ std::optional MaterializeMySQLSyncThread::prepareSynchroniz { std::unique_lock lock(sync_mutex); + bool opened_transaction = false; + mysqlxx::PoolWithFailover::Entry connection; + while (!isCancelled()) { try @@ -210,9 +214,10 @@ std::optional MaterializeMySQLSyncThread::prepareSynchroniz sync_cond.wait_for(lock, std::chrono::seconds(1)); LOG_DEBUG(log, "Database status is OK."); + connection = pool.get(); + opened_transaction = false; - mysqlxx::PoolWithFailover::Entry connection = pool.get(); - MaterializeMetadata metadata(connection, getDatabase(database_name).getMetadataPath() + "/.metadata", mysql_database_name); + MaterializeMetadata metadata(connection, getDatabase(database_name).getMetadataPath() + "/.metadata", mysql_database_name, opened_transaction); if (!metadata.need_dumping_tables.empty()) { @@ -223,16 +228,29 @@ std::optional MaterializeMySQLSyncThread::prepareSynchroniz }); } + if (opened_transaction) + connection->query("COMMIT").execute(); + client.connect(); client.startBinlogDump(std::rand(), mysql_database_name, metadata.binlog_file, metadata.binlog_position); return metadata; } - catch (mysqlxx::Exception & ) + catch (...) { tryLogCurrentException(log); - /// Avoid busy loop when MySQL is not available. - sleepForMilliseconds(settings->max_wait_time_when_mysql_unavailable); + if (opened_transaction) + connection->query("ROLLBACK").execute(); + + try + { + throw; + } + catch (mysqlxx::Exception & ) + { + /// Avoid busy loop when MySQL is not available. + sleepForMilliseconds(settings->max_wait_time_when_mysql_unavailable); + } } } @@ -373,9 +391,20 @@ void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr } else if (receive_event->type() == MYSQL_QUERY_EVENT) { + QueryEvent & query_event = static_cast(*receive_event); flushBuffersData(buffers, metadata); - /// TODO: 执行DDL. - /// TODO: 直接使用Interpreter执行即可 + + try + { + tryToExecuteQuery(query_event.query, global_context, database_name, ""); + } + catch (Exception & exception) + { + tryLogCurrentException(log); + + if (exception.code() != ErrorCodes::SYNTAX_ERROR) + throw; + } } } bool MaterializeMySQLSyncThread::isMySQLSyncThread() @@ -429,12 +458,13 @@ MaterializeMySQLSyncThread::Buffers::BufferAndSortingColumnsPtr MaterializeMySQL { StoragePtr storage = getDatabase(database).tryGetTable(table_name, context); + const StorageInMemoryMetadata & metadata = storage->getInMemoryMetadata(); BufferAndSortingColumnsPtr & buffer_and_soring_columns = data.try_emplace( - table_name, std::make_shared(storage->getSampleBlockNonMaterialized(), std::vector{})).first->second; + table_name, std::make_shared(metadata.getSampleBlockNonMaterialized(), std::vector{})).first->second; if (StorageMergeTree * table_merge_tree = storage->as()) { - Names required_for_sorting_key = table_merge_tree->getColumnsRequiredForSortingKey(); + Names required_for_sorting_key = metadata.getColumnsRequiredForSortingKey(); for (const auto & required_name_for_sorting_key : required_for_sorting_key) buffer_and_soring_columns->second.emplace_back( diff --git a/src/Databases/MySQL/queryConvert.cpp b/src/Databases/MySQL/queryConvert.cpp deleted file mode 100644 index 7e062413523..00000000000 --- a/src/Databases/MySQL/queryConvert.cpp +++ /dev/null @@ -1,156 +0,0 @@ -#include - -#include -#include - -namespace DB -{ - -ASTPtr getFormattedOrderByExpression(const MySQLTableStruct & table_struct) -{ - if (table_struct.primary_keys.empty()) - return makeASTFunction("tuple"); - - /// TODO: support unique key & key - const auto function = std::make_shared(); - function->name = "tuple"; - function->arguments = std::make_shared(); - function->children.push_back(function->arguments); - function->arguments->children = table_struct.primary_keys; - return function; -} - -template -Field choiceBetterRangeSize(TType min, TType max, size_t max_ranges, size_t min_size_pre_range) -{ - UInt64 interval = UInt64(max) - min; - size_t calc_rows_pre_range = std::ceil(interval / double(max_ranges)); - size_t rows_pre_range = std::max(min_size_pre_range, calc_rows_pre_range); - - if (rows_pre_range >= interval) - return Null(); - - return rows_pre_range > std::numeric_limits::max() ? Field(UInt64(rows_pre_range)) : Field(TType(rows_pre_range)); -} - -ASTPtr getFormattedPartitionByExpression(const MySQLTableStruct & table_struct, const Context & context, size_t max_ranges, size_t min_rows_pre_range) -{ - ASTPtr partition_columns = std::make_shared(); - - if (!table_struct.partition_keys.empty()) - partition_columns->children = table_struct.partition_keys; - else if (!table_struct.primary_keys.empty()) - { - ASTPtr expr_list = std::make_shared(); - expr_list->children = table_struct.primary_keys; - - auto syntax = SyntaxAnalyzer(context).analyze(expr_list, table_struct.columns_name_and_type); - auto index_expr = ExpressionAnalyzer(expr_list, syntax, context).getActions(false); - const NamesAndTypesList & required_names_and_types = index_expr->getRequiredColumnsWithTypes(); - - const auto & addPartitionColumn = [&](const String & column_name, const DataTypePtr & type, Field better_pre_range_size) - { - partition_columns->children.emplace_back(std::make_shared(column_name)); - - if (type->isNullable()) - partition_columns->children.back() = makeASTFunction("assumeNotNull", partition_columns->children.back()); - - if (!better_pre_range_size.isNull()) - partition_columns->children.back() - = makeASTFunction("divide", partition_columns->children.back(), std::make_shared(better_pre_range_size)); - }; - - for (const auto & required_name_and_type : required_names_and_types) - { - DataTypePtr assume_not_null = required_name_and_type.type; - if (assume_not_null->isNullable()) - assume_not_null = (static_cast(*assume_not_null)).getNestedType(); - - WhichDataType which(assume_not_null); - if (which.isInt8()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isInt16()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isInt32()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isInt64()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isUInt8()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isUInt16()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isUInt32()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isUInt64()) - addPartitionColumn(required_name_and_type.name, required_name_and_type.type, choiceBetterRangeSize( - std::numeric_limits::min(), std::numeric_limits::max(), max_ranges, min_rows_pre_range)); - else if (which.isDateOrDateTime()) - { - partition_columns->children.emplace_back(std::make_shared(required_name_and_type.name)); - - if (required_name_and_type.type->isNullable()) - partition_columns->children.back() = makeASTFunction("assumeNotNull", partition_columns->children.back()); - - partition_columns->children.back() = makeASTFunction("toYYYYMM", partition_columns->children.back()); - } - } - } - - const auto function = std::make_shared(); - function->name = "tuple"; - function->arguments = partition_columns; - function->children.push_back(function->arguments); - return function; -} - -String getUniqueColumnName(NamesAndTypesList columns_name_and_type, const String & prefix) -{ - const auto & is_unique = [&](const String & column_name) - { - for (const auto & column_name_and_type : columns_name_and_type) - { - if (column_name_and_type.name == column_name) - return false; - } - - return true; - }; - - if (is_unique(prefix)) - return prefix; - - for (size_t index = 0; ; ++index) - { - const String & cur_name = prefix + "_" + toString(index); - if (is_unique(cur_name)) - return cur_name; - } -} - -String toCreateQuery(const MySQLTableStruct & table_struct, const Context & context) -{ - /// TODO: settings - if (table_struct.primary_keys.empty()) - throw Exception("The " + backQuoteIfNeed(table_struct.database_name) + "." + backQuoteIfNeed(table_struct.table_name) - + " cannot be materialized, because there is no primary keys.", ErrorCodes::NOT_IMPLEMENTED); - - WriteBufferFromOwnString out; - String sign = getUniqueColumnName(table_struct.columns_name_and_type, "_sign"); - String version = getUniqueColumnName(table_struct.columns_name_and_type, "_version"); - out << "CREATE TABLE " << (table_struct.if_not_exists ? "IF NOT EXISTS" : "") - << backQuoteIfNeed(table_struct.database_name) + "." << backQuoteIfNeed(table_struct.table_name) - << "(" << queryToString(InterpreterCreateQuery::formatColumns(table_struct.columns_name_and_type)) - << ", " << sign << " Int8, " << version << " UInt64" << ") ENGINE = ReplacingMergeTree(" + version + ")" - << " PARTITION BY " << queryToString(getFormattedPartitionByExpression(table_struct, context, 1000, 50000)) - << " ORDER BY " << queryToString(getFormattedOrderByExpression(table_struct)); - return out.str(); -} - -} diff --git a/src/Databases/MySQL/queryConvert.h b/src/Databases/MySQL/queryConvert.h deleted file mode 100644 index eb5ae5b2648..00000000000 --- a/src/Databases/MySQL/queryConvert.h +++ /dev/null @@ -1,19 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -String toCreateQuery(const MySQLTableStruct & table_struct, const Context & context); - -} diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 6d631c37428..e934e7a5afc 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include @@ -64,11 +65,13 @@ #include #include #include +#include #include #include #include +#include namespace ProfileEvents @@ -242,6 +245,14 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & return std::make_unique(query, context); } else + { + if (MaterializeMySQLSyncThread::isMySQLSyncThread()) + { + if (query->as()) + return std::make_unique(query, context); + } + throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY); + } } } diff --git a/src/Interpreters/MySQL/CreateQueryVisitor.cpp b/src/Interpreters/MySQL/CreateQueryVisitor.cpp index 5090e7eb40f..d2301344872 100644 --- a/src/Interpreters/MySQL/CreateQueryVisitor.cpp +++ b/src/Interpreters/MySQL/CreateQueryVisitor.cpp @@ -75,7 +75,7 @@ void CreateQueryMatcher::visit(const MySQLParser::ASTDeclareColumn & declare_col if (!declare_column.data_type) throw Exception("Missing type in definition of column.", ErrorCodes::UNKNOWN_TYPE); - bool is_nullable = true, is_unsigned = false; + bool is_nullable = true; if (declare_column.column_options) { if (MySQLParser::ASTDeclareOptions * options = declare_column.column_options->as()) @@ -86,10 +86,10 @@ void CreateQueryMatcher::visit(const MySQLParser::ASTDeclareColumn & declare_col if (options->changes.count("primary_key")) data.addPrimaryKey(std::make_shared(declare_column.name)); - if (options->changes.count("is_unsigned")) + /* if (options->changes.count("is_unsigned")) is_unsigned = options->changes["is_unsigned"]->as()->value.safeGet(); else if (options->changes.count("zero_fill")) - is_unsigned = options->changes["zero_fill"]->as()->value.safeGet(); + is_unsigned = options->changes["zero_fill"]->as()->value.safeGet();*/ } } diff --git a/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp b/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp index 160b32b743b..da3c56d9b92 100644 --- a/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp +++ b/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp @@ -1,5 +1,284 @@ -// -// Created by coswde on 2020/6/29. -// +#include -#include "InterpreterMySQLCreateQuery.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_TYPE; + extern const int NOT_IMPLEMENTED; + extern const int EMPTY_LIST_OF_COLUMNS_PASSED; +} + +namespace MySQLInterpreter +{ + +InterpreterMySQLCreateQuery::InterpreterMySQLCreateQuery(const ASTPtr & query_ptr_, Context & context_) + : query_ptr(query_ptr_), context(context_) +{ +} + +BlockIO InterpreterMySQLCreateQuery::execute() +{ + return executeQuery(getRewrittenQuery(), context, true); +} + +static inline NamesAndTypesList getColumnsList(ASTExpressionList * columns_define) +{ + NamesAndTypesList columns_name_and_type; + for (size_t index = 0; index < columns_define->children.size(); ++index) + { + const auto & declare_column = columns_define->children[index]->as(); + + if (!declare_column || !declare_column->data_type) + throw Exception("Missing type in definition of column.", ErrorCodes::UNKNOWN_TYPE); + + bool is_nullable = true; + if (declare_column->column_options) + { + if (const auto * options = declare_column->column_options->as(); + options && options->changes.count("is_null")) + is_nullable = options->changes.at("is_null")->as()->value.safeGet(); + } + + ASTPtr data_type = declare_column->data_type; + + if (is_nullable) + data_type = makeASTFunction("Nullable", data_type); + + columns_name_and_type.emplace_back(declare_column->name, DataTypeFactory::instance().get(data_type)); + } + + return columns_name_and_type; +} + +static NamesAndTypesList getNames(const ASTFunction & expr, const Context & context, const NamesAndTypesList & columns) +{ + if (expr.arguments->children.empty()) + return NamesAndTypesList{}; + + ASTPtr temp_ast = expr.clone(); + auto syntax = SyntaxAnalyzer(context).analyze(temp_ast, columns); + auto expression = ExpressionAnalyzer(temp_ast, syntax, context).getActions(true); + return expression->getRequiredColumnsWithTypes(); +} + +static inline std::tuple getKeys( + ASTExpressionList * columns_define, ASTExpressionList * indices_define, const Context & context, const NamesAndTypesList & columns) +{ + NameSet increment_columns; + auto keys = makeASTFunction("tuple"); + auto unique_keys = makeASTFunction("tuple"); + auto primary_keys = makeASTFunction("tuple"); + + if (indices_define && !indices_define->children.empty()) + { + for (size_t index = 0; index < indices_define->children.size(); ++index) + { + const auto & declare_index = indices_define->children[index]->as(); + + /// flatten + if (startsWith(declare_index->index_type, "KEY_")) + keys->arguments->children.insert(keys->arguments->children.end(), + declare_index->index_columns->children.begin(), declare_index->index_columns->children.end()); + else if (startsWith(declare_index->index_type, "UNIQUE_")) + unique_keys->arguments->children.insert(keys->arguments->children.end(), + declare_index->index_columns->children.begin(), declare_index->index_columns->children.end()); + if (startsWith(declare_index->index_type, "PRIMARY_KEY_")) + primary_keys->arguments->children.insert(keys->arguments->children.end(), + declare_index->index_columns->children.begin(), declare_index->index_columns->children.end()); + } + } + + for (size_t index = 0; index < columns_define->children.size(); ++index) + { + const auto & declare_column = columns_define->children[index]->as(); + + if (declare_column->column_options) + { + if (const auto * options = declare_column->column_options->as()) + { + if (options->changes.count("unique_key")) + unique_keys->arguments->children.emplace_back(std::make_shared(declare_column->name)); + + if (options->changes.count("primary_key")) + primary_keys->arguments->children.emplace_back(std::make_shared(declare_column->name)); + + if (options->changes.contains("auto_increment")) + increment_columns.emplace(declare_column->name); + } + } + } + + return std::make_tuple( + getNames(*primary_keys, context, columns), getNames(*unique_keys, context, columns), getNames(*keys, context, columns), increment_columns); +} + +static String getUniqueColumnName(NamesAndTypesList columns_name_and_type, const String & prefix) +{ + const auto & is_unique = [&](const String & column_name) + { + for (const auto & column_name_and_type : columns_name_and_type) + { + if (column_name_and_type.name == column_name) + return false; + } + + return true; + }; + + if (is_unique(prefix)) + return prefix; + + for (size_t index = 0; ; ++index) + { + const String & cur_name = prefix + "_" + toString(index); + if (is_unique(cur_name)) + return cur_name; + } +} + +static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys) +{ + const auto & numbers_partition = [&](const String & column_name, const DataTypePtr & type, size_t type_max_size) + { + ASTPtr column = std::make_shared(column_name); + + if (type->isNullable()) + column = makeASTFunction("assumeNotNull", column); + + return makeASTFunction("divide", column, std::make_shared(UInt64(type_max_size / 1000))); + }; + + for (const auto & primary_key : primary_keys) + { + WhichDataType which(primary_key.type); + + if (which.isNullable()) + which = WhichDataType((static_cast(*primary_key.type)).getNestedType()); + + if (which.isDateOrDateTime()) + { + ASTPtr res = std::make_shared(primary_key.name); + return makeASTFunction("toYYYYMM", primary_key.type->isNullable() ? makeASTFunction("assumeNotNull", res) : res); + } + + if (which.isInt8() || which.isUInt8()) + return std::make_shared(primary_key.name); + else if (which.isInt16() || which.isUInt16()) + return numbers_partition(primary_key.name, primary_key.type, std::numeric_limits::max()); + else if (which.isInt32() || which.isUInt32()) + return numbers_partition(primary_key.name, primary_key.type, std::numeric_limits::max()); + else if (which.isInt64() || which.isUInt64()) + return numbers_partition(primary_key.name, primary_key.type, std::numeric_limits::max()); + } + + return {}; +} + +static ASTPtr getOrderByPolicy( + const NamesAndTypesList & primary_keys, const NamesAndTypesList & unique_keys, const NamesAndTypesList & keys, const NameSet & increment_columns) +{ + NameSet order_by_columns_set; + std::deque order_by_columns; + + const auto & add_order_by_expression = [&](const NamesAndTypesList & names_and_types) + { + for (const auto & [name, type] : names_and_types) + { + if (order_by_columns_set.contains(name)) + continue; + + if (increment_columns.contains(name)) + { + order_by_columns_set.emplace(name); + order_by_columns.emplace_back(name); + } + else + { + order_by_columns_set.emplace(name); + order_by_columns.emplace_front(name); + } + } + }; + + /// primary_key[not increment], key[not increment], unique[not increment], key[increment], unique[increment], primary_key[increment] + add_order_by_expression(unique_keys); + add_order_by_expression(keys); + add_order_by_expression(primary_keys); + + auto order_by_expression = std::make_shared(); + order_by_expression->name = "tuple"; + order_by_expression->arguments = std::make_shared(); + + for (const auto & order_by_column : order_by_columns) + order_by_expression->arguments->children.emplace_back(std::make_shared(order_by_column)); + + return order_by_expression; +} + +String InterpreterMySQLCreateQuery::getRewrittenQuery() +{ + std::stringstream rewritten_query; + const auto & create_query = query_ptr->as(); + + /// This is dangerous, because the like table may not exists in ClickHouse + if (create_query.like_table) + throw Exception("Cannot convert create like statement to ClickHouse SQL", ErrorCodes::NOT_IMPLEMENTED); + + rewritten_query << "CREATE TABLE " << (create_query.if_not_exists ? "IF NOT EXISTS" : "") << backQuoteIfNeed(create_query.table); + + const auto & create_defines = create_query.columns_list->as(); + + if (!create_defines || !create_defines->columns || create_defines->columns->children.empty()) + throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); + + std::cout << "InterpreterMySQLCreateQuery::getRewrittenQuery() -1 \n"; + NamesAndTypesList columns_name_and_type = getColumnsList(create_defines->columns); + std::cout << "InterpreterMySQLCreateQuery::getRewrittenQuery() -2 \n"; + const auto & [primary_keys, unique_keys, keys, increment_columns] = getKeys(create_defines->columns, create_defines->indices, context, columns_name_and_type); + std::cout << "InterpreterMySQLCreateQuery::getRewrittenQuery() -3 \n"; + + if (primary_keys.empty()) + throw Exception( + "The " + backQuoteIfNeed(create_query.database) + "." + backQuoteIfNeed(create_query.table) + + " cannot be materialized, because there is no primary keys.", + ErrorCodes::NOT_IMPLEMENTED); + + String sign_column_name = getUniqueColumnName(columns_name_and_type, "_sign"); + String version_column_name = getUniqueColumnName(columns_name_and_type, "_version"); + rewritten_query << "(" << queryToString(InterpreterCreateQuery::formatColumns(columns_name_and_type)) + << ", " << sign_column_name << " Int8, " << version_column_name << " UInt64" << ") " + << "ENGINE = ReplacingMergeTree(" + version_column_name + ")"; + + /// The `partition by` expression must use primary keys, otherwise the primary keys will not be merge. + if (ASTPtr partition_expression = getPartitionPolicy(primary_keys)) + rewritten_query << " PARTITION BY " << queryToString(partition_expression); + + /// The `order by` expression must use primary keys, otherwise the primary keys will not be merge. + if (ASTPtr order_by_expression = getOrderByPolicy(primary_keys, unique_keys, keys, increment_columns)) + rewritten_query << " ORDER BY " << queryToString(order_by_expression); + + return rewritten_query.str(); +} + +} + +} diff --git a/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h b/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h index 76bfc2fa2a7..7be8309a728 100644 --- a/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h +++ b/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h @@ -1,12 +1,29 @@ #pragma once +#include +#include + namespace DB { namespace MySQLInterpreter { +/** + */ +class InterpreterMySQLCreateQuery : public IInterpreter +{ +public: + InterpreterMySQLCreateQuery(const ASTPtr & query_ptr_, Context & context_); + BlockIO execute() override; + +private: + ASTPtr query_ptr; + Context & context; + + String getRewrittenQuery(); +}; } diff --git a/src/Parsers/MySQL/ASTDeclareColumn.cpp b/src/Parsers/MySQL/ASTDeclareColumn.cpp index 91dede7a9a7..b4c49b7fe0e 100644 --- a/src/Parsers/MySQL/ASTDeclareColumn.cpp +++ b/src/Parsers/MySQL/ASTDeclareColumn.cpp @@ -1,7 +1,7 @@ #include -#include #include +#include #include #include #include @@ -39,7 +39,6 @@ bool ParserDeclareColumn::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte ASTPtr column_name; ASTPtr column_data_type; ASTPtr column_options; - bool is_national = false; ParserExpression p_expression; ParserIdentifier p_identifier; @@ -47,27 +46,14 @@ bool ParserDeclareColumn::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte if (!p_identifier.parse(pos, column_name, expected)) return false; - if (ParserKeyword("NATIONAL").checkWithoutMoving(pos, expected)) - is_national = true; - else if (ParserKeyword("DOUBLE PRECISION").checkWithoutMoving(pos, expected)) - ParserKeyword("DOUBLE").ignore(pos, expected); /// hack skip DOUBLE - - if (!p_expression.parse(pos, column_data_type, expected)) + if (!ParserDataType().parse(pos, column_data_type, expected)) return false; if (!parseColumnDeclareOptions(pos, column_options, expected)) return false; - if (is_national) - { - if (!column_options) - column_options = std::make_shared(); - column_options->as()->changes.insert( - std::make_pair("is_national", std::make_shared(Field(UInt64(1))))); - } - auto declare_column = std::make_shared(); - declare_column->name = column_name->as()->name; + declare_column->name = getIdentifierName(column_name); declare_column->data_type = column_data_type; declare_column->column_options = column_options; @@ -85,7 +71,6 @@ bool ParserDeclareColumn::parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & ParserDeclareOption p_non_generate_options{ { OptionDescribe("ZEROFILL", "zero_fill", std::make_unique()), - OptionDescribe("UNSIGNED", "is_unsigned", std::make_unique()), OptionDescribe("NULL", "is_null", std::make_unique()), OptionDescribe("NOT NULL", "is_null", std::make_unique()), OptionDescribe("DEFAULT", "default", std::make_unique()), diff --git a/src/Parsers/MySQL/ASTDeclareIndex.cpp b/src/Parsers/MySQL/ASTDeclareIndex.cpp index b48656efd71..ebb269e1361 100644 --- a/src/Parsers/MySQL/ASTDeclareIndex.cpp +++ b/src/Parsers/MySQL/ASTDeclareIndex.cpp @@ -159,7 +159,7 @@ bool ParserDeclareIndex::parseDeclareOrdinaryIndex(IParser::Pos & pos, String & return false; } - index_type = "BTREE"; /// default index type + index_type = "KEY_BTREE"; /// default index type if (p_identifier.parse(pos, temp_node, expected)) index_name = temp_node->as()->name; @@ -168,7 +168,7 @@ bool ParserDeclareIndex::parseDeclareOrdinaryIndex(IParser::Pos & pos, String & if (!p_identifier.parse(pos, temp_node, expected)) return false; - index_type = temp_node->as()->name; + index_type = "KEY_" + temp_node->as()->name; } } diff --git a/src/Parsers/MySQL/ParserMySQLQuery.cpp b/src/Parsers/MySQL/ParserMySQLQuery.cpp index 26bc4effbab..3d565224c88 100644 --- a/src/Parsers/MySQL/ParserMySQLQuery.cpp +++ b/src/Parsers/MySQL/ParserMySQLQuery.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -15,7 +16,7 @@ namespace MySQLParser bool ParserMySQLQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { - if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) + if (getThreadName() != "MySQLDBSync") return false; ParserDropQuery p_drop_query; diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index 03a1d511d26..5d4ec84ecf5 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -17,18 +17,21 @@ StorageMaterializeMySQL::StorageMaterializeMySQL(const StoragePtr & nested_stora : IStorage(nested_storage_->getStorageID()), nested_storage(nested_storage_) { ColumnsDescription columns_desc; - const ColumnsDescription & nested_columns_desc = nested_storage->getColumns(); + const auto & nested_memory_metadata = nested_storage->getInMemoryMetadata(); + const ColumnsDescription & nested_columns_desc = nested_memory_metadata.getColumns(); - size_t index = 0; auto iterator = nested_columns_desc.begin(); - for (; index < nested_columns_desc.size() - 2; ++index, ++iterator) + for (size_t index = 0; index < nested_columns_desc.size() - 2; ++index, ++iterator) columns_desc.add(*iterator); - setColumns(columns_desc); + StorageInMemoryMetadata in_memory_metadata; + in_memory_metadata.setColumns(columns_desc); + setInMemoryMetadata(in_memory_metadata); } Pipes StorageMaterializeMySQL::read( const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -36,8 +39,9 @@ Pipes StorageMaterializeMySQL::read( unsigned int num_streams) { NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); + const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); - Block nested_header = nested_storage->getSampleBlockNonMaterialized(); + Block nested_header = nested_metadata->getSampleBlockNonMaterialized(); ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); @@ -71,7 +75,7 @@ Pipes StorageMaterializeMySQL::read( expressions->children.emplace_back(std::make_shared(column_name)); } - Pipes pipes = nested_storage->read(require_columns_name, query_info, context, processed_stage, max_block_size, num_streams); + Pipes pipes = nested_storage->read(require_columns_name, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); if (!expressions->children.empty() && !pipes.empty()) { @@ -92,7 +96,7 @@ Pipes StorageMaterializeMySQL::read( NamesAndTypesList StorageMaterializeMySQL::getVirtuals() const { NamesAndTypesList virtuals; - Block nested_header = nested_storage->getSampleBlockNonMaterialized(); + Block nested_header = nested_storage->getInMemoryMetadata().getSampleBlockNonMaterialized(); ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); virtuals.emplace_back(NameAndTypePair(sign_column.name, sign_column.type)); diff --git a/src/Storages/StorageMaterializeMySQL.h b/src/Storages/StorageMaterializeMySQL.h index a74f2208a9a..e1b6aa12479 100644 --- a/src/Storages/StorageMaterializeMySQL.h +++ b/src/Storages/StorageMaterializeMySQL.h @@ -18,7 +18,7 @@ public: StorageMaterializeMySQL(const StoragePtr & nested_storage_); Pipes read( - const Names & column_names, const SelectQueryInfo & query_info, + const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; NamesAndTypesList getVirtuals() const override; From 1583f67681b325a18b184f9027886b222f4561eb Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 1 Jul 2020 13:56:24 +0800 Subject: [PATCH 078/374] ISSUES-4006 remove unused code --- .../MySQL/MaterializeMySQLSyncThread.cpp | 1 - .../MySQL/MaterializeMySQLSyncThread.h | 1 - src/Interpreters/MySQL/CreateQueryVisitor.cpp | 178 ----------------- src/Interpreters/MySQL/CreateQueryVisitor.h | 78 -------- .../MySQL/InterpreterMySQLCreateQuery.cpp | 44 +++-- .../MySQL/InterpreterMySQLCreateQuery.h | 2 +- .../gtest_create_query_convert_visitor.cpp | 187 ------------------ 7 files changed, 26 insertions(+), 465 deletions(-) delete mode 100644 src/Interpreters/MySQL/CreateQueryVisitor.cpp delete mode 100644 src/Interpreters/MySQL/CreateQueryVisitor.h delete mode 100644 src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 69576df42ca..f96461af427 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -16,7 +16,6 @@ # include # include # include -# include # include # include # include diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index 38b59fbb736..006147a6cd7 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -14,7 +14,6 @@ # include # include # include -# include # include # include # include diff --git a/src/Interpreters/MySQL/CreateQueryVisitor.cpp b/src/Interpreters/MySQL/CreateQueryVisitor.cpp deleted file mode 100644 index d2301344872..00000000000 --- a/src/Interpreters/MySQL/CreateQueryVisitor.cpp +++ /dev/null @@ -1,178 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int UNKNOWN_TYPE; - extern const int NOT_IMPLEMENTED; - extern const int EMPTY_LIST_OF_COLUMNS_PASSED; -} - -namespace MySQLVisitor -{ - -void CreateQueryMatcher::visit(ASTPtr & ast, Data & data) -{ - if (auto * t = ast->as()) - visit(*t, ast, data); -} - -void CreateQueryMatcher::visit(const MySQLParser::ASTCreateQuery & create, const ASTPtr &, Data & data) -{ - if (create.like_table) - throw Exception("Cannot convert create like statement to ClickHouse SQL", ErrorCodes::NOT_IMPLEMENTED); - - if (create.columns_list) - visit(*create.columns_list->as(), create.columns_list, data); - - if (create.partition_options) - visit(*create.partition_options->as(), create.partition_options, data); - - data.table_name = create.table; - data.database_name = create.database; - data.if_not_exists = create.if_not_exists; -} - -void CreateQueryMatcher::visit(const MySQLParser::ASTDeclareIndex & declare_index, const ASTPtr &, Data & data) -{ - if (startsWith(declare_index.index_type, "PRIMARY_KEY_")) - data.addPrimaryKey(declare_index.index_columns); -} - -void CreateQueryMatcher::visit(const MySQLParser::ASTCreateDefines & create_defines, const ASTPtr &, Data & data) -{ - if (!create_defines.columns || create_defines.columns->children.empty()) - throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); - - if (create_defines.indices) - { - for (auto & index : create_defines.indices->children) - visit(*index->as(), index, data); - } - - for (auto & column : create_defines.columns->children) - visit(*column->as(), column, data); -} - -void CreateQueryMatcher::visit(const MySQLParser::ASTDeclareColumn & declare_column, const ASTPtr &, Data & data) -{ - if (!declare_column.data_type) - throw Exception("Missing type in definition of column.", ErrorCodes::UNKNOWN_TYPE); - - bool is_nullable = true; - if (declare_column.column_options) - { - if (MySQLParser::ASTDeclareOptions * options = declare_column.column_options->as()) - { - if (options->changes.count("is_null")) - is_nullable = options->changes["is_null"]->as()->value.safeGet(); - - if (options->changes.count("primary_key")) - data.addPrimaryKey(std::make_shared(declare_column.name)); - - /* if (options->changes.count("is_unsigned")) - is_unsigned = options->changes["is_unsigned"]->as()->value.safeGet(); - else if (options->changes.count("zero_fill")) - is_unsigned = options->changes["zero_fill"]->as()->value.safeGet();*/ - } - } - - data.columns_name_and_type.emplace_back(declare_column.name, DataTypeFactory::instance().get( - (is_nullable ? "Nullable(" : "") + queryToString(declare_column.data_type) + (is_nullable ? ")" : ""))); -} - -void CreateQueryMatcher::visit(const MySQLParser::ASTDeclarePartitionOptions & declare_partition_options, const ASTPtr &, Data & data) -{ - data.addPartitionKey(declare_partition_options.partition_expression); -} - -void CreateQueryMatcher::Data::addPrimaryKey(const ASTPtr & primary_key) -{ - if (const auto & function_index_columns = primary_key->as()) - { - if (function_index_columns->name != "tuple") - throw Exception("Unable to parse function primary key from MySQL.", ErrorCodes::NOT_IMPLEMENTED); - - for (const auto & index_column : function_index_columns->arguments->children) - primary_keys.emplace_back(index_column); - } - else if (const auto & expression_index_columns = primary_key->as()) - { - for (const auto & index_column : expression_index_columns->children) - primary_keys.emplace_back(index_column); - } - else - primary_keys.emplace_back(primary_key); -} - -void CreateQueryMatcher::Data::addPartitionKey(const ASTPtr & partition_key) -{ - if (const auto & function_partition_columns = partition_key->as()) - { - if (function_partition_columns->name != "tuple") - throw Exception("Unable to parse function partition by from MySQL.", ErrorCodes::NOT_IMPLEMENTED); - - for (const auto & partition_column : function_partition_columns->arguments->children) - partition_keys.emplace_back(partition_column); - } - else if (const auto & expression_partition_columns = partition_key->as()) - { - for (const auto & partition_column : expression_partition_columns->children) - partition_keys.emplace_back(partition_column); - } - else - partition_keys.emplace_back(partition_key); -} - -} - -bool MySQLTableStruct::operator==(const MySQLTableStruct & other) const -{ - const auto & this_expression = std::make_shared(); - this_expression->children.insert(this_expression->children.begin(), primary_keys.begin(), primary_keys.end()); - this_expression->children.insert(this_expression->children.begin(), partition_keys.begin(), partition_keys.end()); - - const auto & other_expression = std::make_shared(); - other_expression->children.insert(other_expression->children.begin(), other.primary_keys.begin(), other.primary_keys.end()); - other_expression->children.insert(other_expression->children.begin(), other.partition_keys.begin(), other.partition_keys.end()); - - return queryToString(this_expression) == queryToString(other_expression) && columns_name_and_type == other.columns_name_and_type; -} - -MySQLTableStruct visitCreateQuery(ASTPtr & create_query, const Context & context, const std::string & new_database) -{ - create_query->as()->database = new_database; - MySQLVisitor::CreateQueryVisitor::Data table_struct(context); - MySQLVisitor::CreateQueryVisitor visitor(table_struct); - visitor.visit(create_query); - return std::move(table_struct); -} - -MySQLTableStruct visitCreateQuery(const String & create_query, const Context & context, const std::string & new_database) -{ - MySQLParser::ParserCreateQuery p_create_query; - ASTPtr ast_create_query = parseQuery(p_create_query, create_query.data(), create_query.data() + create_query.size(), "", 0, 0); - - if (!ast_create_query || !ast_create_query->as()) - throw Exception("LOGICAL ERROR: ast cannot cast to MySQLParser::ASTCreateQuery.", ErrorCodes::LOGICAL_ERROR); - - return visitCreateQuery(ast_create_query, context, new_database); -} - -} diff --git a/src/Interpreters/MySQL/CreateQueryVisitor.h b/src/Interpreters/MySQL/CreateQueryVisitor.h deleted file mode 100644 index 2a57ab817c1..00000000000 --- a/src/Interpreters/MySQL/CreateQueryVisitor.h +++ /dev/null @@ -1,78 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -struct MySQLTableStruct -{ - bool if_not_exists; - String table_name; - String database_name; - ASTs primary_keys; - ASTs partition_keys; - NamesAndTypesList columns_name_and_type; - - MySQLTableStruct() {} - - MySQLTableStruct(const ASTs & primary_keys_, const ASTs & partition_keys_, const NamesAndTypesList & columns_name_and_type_) - : primary_keys(primary_keys_), partition_keys(partition_keys_), columns_name_and_type(columns_name_and_type_) - {} - - bool operator==(const MySQLTableStruct & other) const; -}; - -namespace MySQLVisitor -{ - -/// Convert MySQL CREATE query to https://clickhouse.tech/docs/en/sql-reference/statements/create/ -class CreateQueryMatcher -{ -public: - using Visitor = InDepthNodeVisitor; - - struct Data : public MySQLTableStruct - { - const Context & context; - size_t max_ranges; - size_t min_rows_pre_range; - - Data(const Context & context_) : MySQLTableStruct(), context(context_) {} - - void addPrimaryKey(const ASTPtr & primary_key); - - void addPartitionKey(const ASTPtr & partition_key); - }; - - static void visit(ASTPtr & ast, Data & data); - - static bool needChildVisit(ASTPtr &, const ASTPtr &) { return false; } -private: - static void visit(const MySQLParser::ASTCreateQuery & create, const ASTPtr &, Data & data); - - static void visit(const MySQLParser::ASTDeclareIndex & declare_index, const ASTPtr &, Data & data); - - static void visit(const MySQLParser::ASTCreateDefines & create_defines, const ASTPtr &, Data & data); - - static void visit(const MySQLParser::ASTDeclareColumn & declare_column, const ASTPtr &, Data & data); - - static void visit(const MySQLParser::ASTDeclarePartitionOptions & declare_partition_options, const ASTPtr &, Data & data); -}; - -using CreateQueryVisitor = CreateQueryMatcher::Visitor; - -} - -MySQLTableStruct visitCreateQuery(ASTPtr & create_query, const Context & context, const std::string & new_database); - -MySQLTableStruct visitCreateQuery(const String & create_query, const Context & context, const std::string & new_database); - -} diff --git a/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp b/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp index da3c56d9b92..3d273d32772 100644 --- a/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp +++ b/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp @@ -4,13 +4,14 @@ #include #include #include -#include +#include #include #include #include #include #include +#include #include #include #include @@ -37,7 +38,7 @@ InterpreterMySQLCreateQuery::InterpreterMySQLCreateQuery(const ASTPtr & query_pt BlockIO InterpreterMySQLCreateQuery::execute() { - return executeQuery(getRewrittenQuery(), context, true); + return InterpreterCreateQuery(getRewrittenQuery(), context).execute(); } static inline NamesAndTypesList getColumnsList(ASTExpressionList * columns_define) @@ -234,49 +235,54 @@ static ASTPtr getOrderByPolicy( return order_by_expression; } -String InterpreterMySQLCreateQuery::getRewrittenQuery() +ASTPtr InterpreterMySQLCreateQuery::getRewrittenQuery() { - std::stringstream rewritten_query; + auto rewritten_query = std::make_shared(); const auto & create_query = query_ptr->as(); /// This is dangerous, because the like table may not exists in ClickHouse if (create_query.like_table) throw Exception("Cannot convert create like statement to ClickHouse SQL", ErrorCodes::NOT_IMPLEMENTED); - rewritten_query << "CREATE TABLE " << (create_query.if_not_exists ? "IF NOT EXISTS" : "") << backQuoteIfNeed(create_query.table); - const auto & create_defines = create_query.columns_list->as(); if (!create_defines || !create_defines->columns || create_defines->columns->children.empty()) throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); - std::cout << "InterpreterMySQLCreateQuery::getRewrittenQuery() -1 \n"; NamesAndTypesList columns_name_and_type = getColumnsList(create_defines->columns); - std::cout << "InterpreterMySQLCreateQuery::getRewrittenQuery() -2 \n"; const auto & [primary_keys, unique_keys, keys, increment_columns] = getKeys(create_defines->columns, create_defines->indices, context, columns_name_and_type); - std::cout << "InterpreterMySQLCreateQuery::getRewrittenQuery() -3 \n"; if (primary_keys.empty()) - throw Exception( - "The " + backQuoteIfNeed(create_query.database) + "." + backQuoteIfNeed(create_query.table) - + " cannot be materialized, because there is no primary keys.", - ErrorCodes::NOT_IMPLEMENTED); + throw Exception("The " + backQuoteIfNeed(create_query.database) + "." + backQuoteIfNeed(create_query.table) + + " cannot be materialized, because there is no primary keys.", ErrorCodes::NOT_IMPLEMENTED); + auto columns = std::make_shared(); + + /// Add _sign and _version column. String sign_column_name = getUniqueColumnName(columns_name_and_type, "_sign"); String version_column_name = getUniqueColumnName(columns_name_and_type, "_version"); - rewritten_query << "(" << queryToString(InterpreterCreateQuery::formatColumns(columns_name_and_type)) - << ", " << sign_column_name << " Int8, " << version_column_name << " UInt64" << ") " - << "ENGINE = ReplacingMergeTree(" + version_column_name + ")"; + columns_name_and_type.emplace_back(NameAndTypePair{sign_column_name, std::make_shared()}); + columns_name_and_type.emplace_back(NameAndTypePair{version_column_name, std::make_shared()}); + columns->set(columns->columns, InterpreterCreateQuery::formatColumns(columns_name_and_type)); + + auto storage = std::make_shared(); /// The `partition by` expression must use primary keys, otherwise the primary keys will not be merge. if (ASTPtr partition_expression = getPartitionPolicy(primary_keys)) - rewritten_query << " PARTITION BY " << queryToString(partition_expression); + storage->set(storage->partition_by, partition_expression); /// The `order by` expression must use primary keys, otherwise the primary keys will not be merge. if (ASTPtr order_by_expression = getOrderByPolicy(primary_keys, unique_keys, keys, increment_columns)) - rewritten_query << " ORDER BY " << queryToString(order_by_expression); + storage->set(storage->order_by, order_by_expression); - return rewritten_query.str(); + storage->set(storage->engine, makeASTFunction("ReplacingMergeTree", std::make_shared(version_column_name))); + + rewritten_query->table = create_query.table; + rewritten_query->if_not_exists = create_query.if_not_exists; + rewritten_query->set(rewritten_query->storage, storage); + rewritten_query->set(rewritten_query->columns_list, columns); + + return rewritten_query; } } diff --git a/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h b/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h index 7be8309a728..22f71501e81 100644 --- a/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h +++ b/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h @@ -22,7 +22,7 @@ private: ASTPtr query_ptr; Context & context; - String getRewrittenQuery(); + ASTPtr getRewrittenQuery(); }; } diff --git a/src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp b/src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp deleted file mode 100644 index d44c513fea6..00000000000 --- a/src/Interpreters/MySQL/tests/gtest_create_query_convert_visitor.cpp +++ /dev/null @@ -1,187 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include - -using namespace DB; -using namespace MySQLParser; -using namespace MySQLVisitor; - -static DataTypePtr getType(const String & data_type) -{ - return DataTypeFactory::instance().get(data_type); -} - -static ContextShared * contextShared() -{ - static SharedContextHolder shared = Context::createShared(); - return shared.get(); -} - -static MySQLTableStruct visitQuery(const String & query) -{ - ParserCreateQuery p_create_query; - ASTPtr ast = parseQuery(p_create_query, query.data(), query.data() + query.size(), "", 0, 0); - - CreateQueryVisitor::Data data(Context::createGlobal(contextShared())); - data.max_ranges = 1000; - data.min_rows_pre_range = 1000000; - CreateQueryVisitor visitor(data); - visitor.visit(ast); - return std::move(data); -} - -TEST(CreateQueryVisitor, TestWithNumberColumnsType) -{ - EXPECT_EQ( - visitQuery("CREATE TABLE test(a tinyint, b SMALLINT, c MEDIUMINT, d INT, e INTEGER, f BIGINT, g DECIMAL, h DEC, i NUMERIC, j " - "FIXED, k FLOAT, l DOUBLE, m DOUBLE PRECISION, n REAL)"), - MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Nullable(Int8)")}, {"b", getType("Nullable(Int16)")} - , {"c", getType("Nullable(Int32)")}, {"d", getType("Nullable(Int32)")}, {"e", getType("Nullable(Int32)")} - , {"f", getType("Nullable(Int64)")}, {"g", getType("Nullable(Decimal(10, 0))")}, {"h", getType("Nullable(Decimal(10, 0))")} - , {"i", getType("Nullable(Decimal(10, 0))")}, {"j", getType("Nullable(Decimal(10, 0))")} - , {"k", getType("Nullable(Float32)")}, {"l", getType("Nullable(Float64)")}, {"m", getType("Nullable(Float64)")} - , {"n", getType("Nullable(Float64)")}} - ) - ); - - EXPECT_EQ( - visitQuery("CREATE TABLE test(a tinyint(1), b SMALLINT(1), c MEDIUMINT(1), d INT(1), e INTEGER(1), f BIGINT(1), g DECIMAL(1), h " - "DEC(2, 1), i NUMERIC(4, 3), j FIXED(6, 5), k FLOAT(1), l DOUBLE(1, 2), m DOUBLE PRECISION(3, 4), n REAL(5, 6))"), - MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Nullable(Int8)")}, {"b", getType("Nullable(Int16)")} - , {"c", getType("Nullable(Int32)")}, {"d", getType("Nullable(Int32)")}, {"e", getType("Nullable(Int32)")} - , {"f", getType("Nullable(Int64)")}, {"g", getType("Nullable(Decimal(1, 0))")}, {"h", getType("Nullable(Decimal(2, 1))")} - , {"i", getType("Nullable(Decimal(4, 3))")}, {"j", getType("Nullable(Decimal(6, 5))")} - , {"k", getType("Nullable(Float32)")}, {"l", getType("Nullable(Float64)")}, {"m", getType("Nullable(Float64)")} - , {"n", getType("Nullable(Float64)")}} - ) - ); - - /// UNSIGNED - EXPECT_EQ( - visitQuery("CREATE TABLE test(a tinyint UNSIGNED, b SMALLINT(1) UNSIGNED, c MEDIUMINT(1) UNSIGNED, d INT(1) UNSIGNED, e INTEGER(1), f " - "BIGINT(1) UNSIGNED, g DECIMAL(1) UNSIGNED, h DEC(2, 1) UNSIGNED, i NUMERIC(4, 3) UNSIGNED, j FIXED(6, 5) UNSIGNED, k FLOAT(1) " - "UNSIGNED, l DOUBLE(1, 2) UNSIGNED, m DOUBLE PRECISION(3, 4) UNSIGNED, n REAL(5, 6) UNSIGNED)"), - MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Nullable(UInt8)")}, {"b", getType("Nullable(UInt16)")} - , {"c", getType("Nullable(UInt32)")}, {"d", getType("Nullable(UInt32)")}, {"e", getType("Nullable(Int32)")} - , {"f", getType("Nullable(UInt64)")}, {"g", getType("Nullable(Decimal(1, 0))")}, {"h", getType("Nullable(Decimal(2, 1))")} - , {"i", getType("Nullable(Decimal(4, 3))")}, {"j", getType("Nullable(Decimal(6, 5))")} - , {"k", getType("Nullable(Float32)")}, {"l", getType("Nullable(Float64)")}, {"m", getType("Nullable(Float64)")} - , {"n", getType("Nullable(Float64)")}} - ) - ); - - /// NOT NULL - EXPECT_EQ( - visitQuery("CREATE TABLE test(a tinyint NOT NULL, b SMALLINT(1) NOT NULL, c MEDIUMINT(1) NOT NULL, d INT(1) NOT NULL, e INTEGER(1), f " - "BIGINT(1) NOT NULL, g DECIMAL(1) NOT NULL, h DEC(2, 1) NOT NULL, i NUMERIC(4, 3) NOT NULL, j FIXED(6, 5) NOT NULL, k FLOAT(1) NOT " - "NULL, l DOUBLE(1, 2) NOT NULL, m DOUBLE PRECISION(3, 4) NOT NULL, n REAL(5, 6) NOT NULL)"), - MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Int8")}, {"b", getType("Int16")} - , {"c", getType("Int32")}, {"d", getType("Int32")}, {"e", getType("Nullable(Int32)")} - , {"f", getType("Int64")}, {"g", getType("Decimal(1, 0)")}, {"h", getType("Decimal(2, 1)")} - , {"i", getType("Decimal(4, 3)")}, {"j", getType("Decimal(6, 5)")} - , {"k", getType("Float32")}, {"l", getType("Float64")}, {"m", getType("Float64")}, {"n", getType("Float64")}} - ) - ); - - /// ZEROFILL - EXPECT_EQ( - visitQuery("CREATE TABLE test(a tinyint ZEROFILL, b SMALLINT(1) ZEROFILL, c MEDIUMINT(1) ZEROFILL, d INT(1) ZEROFILL, e INTEGER(1), f " - "BIGINT(1) ZEROFILL, g DECIMAL(1) ZEROFILL, h DEC(2, 1) ZEROFILL, i NUMERIC(4, 3) ZEROFILL, j FIXED(6, 5) ZEROFILL, k FLOAT(1) " - "ZEROFILL, l DOUBLE(1, 2) ZEROFILL, m DOUBLE PRECISION(3, 4) ZEROFILL, n REAL(5, 6) ZEROFILL)"), - MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Nullable(UInt8)")}, {"b", getType("Nullable(UInt16)")} - , {"c", getType("Nullable(UInt32)")}, {"d", getType("Nullable(UInt32)")}, {"e", getType("Nullable(Int32)")} - , {"f", getType("Nullable(UInt64)")}, {"g", getType("Nullable(Decimal(1, 0))")}, {"h", getType("Nullable(Decimal(2, 1))")} - , {"i", getType("Nullable(Decimal(4, 3))")}, {"j", getType("Nullable(Decimal(6, 5))")} - , {"k", getType("Nullable(Float32)")}, {"l", getType("Nullable(Float64)")}, {"m", getType("Nullable(Float64)")} - , {"n", getType("Nullable(Float64)")}} - ) - ); -} - -TEST(CreateQueryVisitor, TestWithDateTimesColumnsType) -{ - EXPECT_EQ( - visitQuery("CREATE TABLE test(a DATE, b DATETIME, c TIMESTAMP, d TIME, e year)"), - MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Nullable(Date)")}, {"b", getType("Nullable(DateTime)")} - , {"c", getType("Nullable(DateTime)")}, {"d", getType("Nullable(DateTime64(3))")}, {"e", getType("Nullable(Int16)")} } - ) - ); - - EXPECT_EQ( - visitQuery("CREATE TABLE test(a DATE, b DATETIME(1), c TIMESTAMP(1), d TIME(1), e year(4))"), - MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Nullable(Date)")}, {"b", getType("Nullable(DateTime)")} - , {"c", getType("Nullable(DateTime)")}, {"d", getType("Nullable(DateTime64(3))")}, {"e", getType("Nullable(Int16)")} } - ) - ); - - EXPECT_EQ( - visitQuery("CREATE TABLE test(a DATE NOT NULL, b DATETIME(1) NOT NULL, c TIMESTAMP(1) NOT NULL, d TIME(1) NOT NULL, e year(4) NOT NULL)"), - MySQLTableStruct(ASTs{}, ASTs{}, {{"a", getType("Date")}, {"b", getType("DateTime")} , {"c", getType("DateTime")}, {"d", getType("DateTime64")}, {"e", getType("Int16")} } - ) - ); -} - -TEST(CreateQueryVisitor, TestWithParitionOptions) -{ - EXPECT_EQ( - visitQuery("CREATE TABLE test(a DATE NOT NULL) PARTITION BY HASH a"), - MySQLTableStruct(ASTs{}, ASTs{std::make_shared("a")}, {{"a", getType("Date")}})); - - EXPECT_EQ( - visitQuery("CREATE TABLE test(a DATE NOT NULL) PARTITION BY LINEAR HASH a"), - MySQLTableStruct(ASTs{}, ASTs{std::make_shared("a")}, {{"a", getType("Date")}})); - - EXPECT_EQ( - visitQuery("CREATE TABLE test(a DATE NOT NULL) PARTITION BY RANGE(a)"), - MySQLTableStruct(ASTs{}, ASTs{std::make_shared("a")}, {{"a", getType("Date")}})); - - EXPECT_EQ( - visitQuery("CREATE TABLE test(a DATE NOT NULL, b INT) PARTITION BY RANGE COLUMNS(a, b)"), - MySQLTableStruct(ASTs{}, ASTs{std::make_shared("a"), std::make_shared("b")}, {{"a", getType("Date")}, {"b", getType("Nullable(Int32)")}})); - - EXPECT_EQ( - visitQuery("CREATE TABLE test(a DATE NOT NULL) PARTITION BY LIST(a)"), - MySQLTableStruct(ASTs{}, ASTs{std::make_shared("a")}, {{"a", getType("Date")}})); - - EXPECT_EQ( - visitQuery("CREATE TABLE test(a DATE NOT NULL, b INT) PARTITION BY LIST COLUMNS(a, b)"), - MySQLTableStruct(ASTs{}, ASTs{std::make_shared("a"), std::make_shared("b")}, - {{"a", getType("Date")}, {"b", getType("Nullable(Int32)")}})); -} - -TEST(CreateQueryVisitor, TestWithPrimaryToPartitionBy) -{ - EXPECT_EQ( - visitQuery("CREATE TABLE test(a DATE NOT NULL PRIMARY KEY)"), - MySQLTableStruct(ASTs{std::make_shared("a")}, ASTs{}, {{"a", getType("Date")}})); - - EXPECT_EQ( - visitQuery("CREATE TABLE test(a DATETIME NOT NULL PRIMARY KEY)"), - MySQLTableStruct(ASTs{std::make_shared("a")}, ASTs{}, {{"a", getType("DateTime")}})); - - EXPECT_EQ( - visitQuery("CREATE TABLE test(a TINYINT NOT NULL PRIMARY KEY)"), - MySQLTableStruct(ASTs{std::make_shared("a")}, ASTs{}, {{"a", getType("Int8")}})); - - EXPECT_EQ( - visitQuery("CREATE TABLE test(a SMALLINT NOT NULL PRIMARY KEY)"), - MySQLTableStruct(ASTs{std::make_shared("a")}, ASTs{}, {{"a", getType("Int16")}})); - - EXPECT_EQ( - visitQuery("CREATE TABLE test(a INT NOT NULL PRIMARY KEY)"), - MySQLTableStruct(ASTs{std::make_shared("a")}, ASTs{}, {{"a", getType("Int32")}})); - - EXPECT_EQ( - visitQuery("CREATE TABLE test(a BIGINT NOT NULL PRIMARY KEY)"), - MySQLTableStruct(ASTs{std::make_shared("a")}, ASTs{}, {{"a", getType("Int64")}})); - - EXPECT_EQ( - visitQuery("CREATE TABLE test(a BIGINT PRIMARY KEY)"), - MySQLTableStruct(ASTs{std::make_shared("a")}, ASTs{}, {{"a", getType("Nullable(Int64)")}})); -} - From 656259706451dd02d42f99a9b297c416631f19eb Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 1 Jul 2020 14:42:22 +0800 Subject: [PATCH 079/374] ISSUES-4006 refactor interpreter --- src/Core/MySQLClient.cpp | 3 +- src/Core/MySQLReplication.cpp | 23 ++---- .../MySQL/MaterializeMySQLSyncThread.cpp | 36 ++++----- .../MySQL/MaterializeMySQLSyncThread.h | 1 + src/Interpreters/InterpreterFactory.cpp | 20 ++--- .../MySQL/InterpreterMySQLCreateQuery.h | 30 -------- ...eQuery.cpp => MySQLInterpreterFactory.cpp} | 28 ++++--- .../MySQL/MySQLInterpreterFactory.h | 59 ++++++++++++++ src/Interpreters/executeQuery.cpp | 18 ++++- src/Interpreters/executeQuery.h | 8 ++ src/Parsers/MySQL/ASTAlterQuery.cpp | 76 +++++++++++++++++++ src/Parsers/MySQL/ASTAlterQuery.h | 75 ++++++++++++++++++ src/Parsers/MySQL/ParserMySQLQuery.cpp | 4 - src/Parsers/MySQL/ParserMySQLQuery.h | 3 + src/Parsers/ParserExternalDDLQuery.cpp | 5 ++ src/Parsers/ParserExternalDDLQuery.h | 6 ++ src/Parsers/ParserQuery.cpp | 5 +- 17 files changed, 302 insertions(+), 98 deletions(-) delete mode 100644 src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h rename src/Interpreters/MySQL/{InterpreterMySQLCreateQuery.cpp => MySQLInterpreterFactory.cpp} (93%) create mode 100644 src/Interpreters/MySQL/MySQLInterpreterFactory.h create mode 100644 src/Parsers/MySQL/ASTAlterQuery.cpp create mode 100644 src/Parsers/MySQL/ASTAlterQuery.h create mode 100644 src/Parsers/ParserExternalDDLQuery.cpp create mode 100644 src/Parsers/ParserExternalDDLQuery.h diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 109e8e1a7da..5cd71377a68 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -152,7 +152,8 @@ void MySQLClient::startBinlogDump(UInt32 slave_id, String replicate_db, String b try { packet_sender->receivePacket(replication); - events.push(std::make_pair(replication.readOneEvent(), replication.getPosition())); + auto receive_event = replication.readOneEvent(); + events.push(std::make_pair(receive_event, replication.getPosition())); } catch(...) { diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index e73a920ac88..d31c4c0b128 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -757,24 +757,17 @@ namespace MySQLReplication event->parseEvent(payload); auto query = std::static_pointer_cast(event); - if (query->schema == replicate_do_db) + switch (query->typ) { - switch (query->typ) - { - case BEGIN: - case XA: { - event = std::make_shared(); - break; - } - default: - position.updateLogPos(event->header.log_pos); + case BEGIN: + case XA: { + event = std::make_shared(); + break; } + default: + position.updateLogPos(event->header.log_pos); } - else - { - event = std::make_shared(); - position.updateLogPos(event->header.log_pos); - } + break; } case XID_EVENT: { diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index f96461af427..d629cab977a 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -7,22 +7,20 @@ #include # include +# include +# include +# include # include -# include -# include # include +# include +# include # include # include # include # include # include # include -# include # include -# include -# include -# include -# include namespace DB { @@ -35,18 +33,21 @@ namespace ErrorCodes static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; +template static BlockIO tryToExecuteQuery(const String & query_to_execute, const Context & context_, const String & database, const String & comment) { try { - LOG_DEBUG(&Poco::Logger::get("MaterializeMySQLSyncThread(" + database + ")"), "Try execute query: " + query_to_execute); - Context context(context_); CurrentThread::QueryScope query_scope(context); context.setCurrentDatabase(database); context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; context.setCurrentQueryId(""); // generate random query_id - return executeQuery("/*" + comment + "*/ " + query_to_execute, context, true); + + if constexpr (execute_ddl) + return executeMySQLDDLQuery("/*" + comment + "*/ " + query_to_execute, context, true); + else + return executeQuery("/*" + comment + "*/ " + query_to_execute, context, true); } catch (...) { @@ -55,8 +56,6 @@ static BlockIO tryToExecuteQuery(const String & query_to_execute, const Context "Query " + query_to_execute + " wasn't finished successfully"); throw; } - - LOG_DEBUG(&Poco::Logger::get("MaterializeMySQLSyncThread(" + database + ")"), "Executed query: " + query_to_execute); } static inline DatabaseMaterializeMySQL & getDatabase(const String & database_name) @@ -97,6 +96,7 @@ MaterializeMySQLSyncThread::MaterializeMySQLSyncThread( , mysql_database_name(mysql_database_name_), pool(std::move(pool_)), client(std::move(client_)), settings(settings_) { /// TODO: 做简单的check, 失败即报错 + query_prefix = "EXTERNAL DDL FROM MySQL(" + backQuoteIfNeed(database_name) + ", " + backQuoteIfNeed(mysql_database_name) + ")"; startSynchronization(); } @@ -162,14 +162,15 @@ static inline void cleanOutdatedTables(const String & database_name, const Conte for (auto iterator = clean_database->getTablesIterator(context); iterator->isValid(); iterator->next()) { String table_name = backQuoteIfNeed(iterator->name()); - String comment = String("Clean ") + table_name + " for dump mysql."; - tryToExecuteQuery("DROP TABLE " + table_name, context, backQuoteIfNeed(database_name), comment); + String comment = "Materialize MySQL step 1: execute MySQL DDL for dump data"; + tryToExecuteQuery("DROP TABLE " + table_name, context, database_name, comment); } } static inline BlockOutputStreamPtr getTableOutput(const String & database_name, const String & table_name, const Context & context) { - BlockIO res = tryToExecuteQuery("INSERT INTO " + backQuoteIfNeed(table_name) + " VALUES", context, database_name, ""); + String comment = "Materialize MySQL step 1: execute dump data"; + BlockIO res = tryToExecuteQuery("INSERT INTO " + backQuoteIfNeed(table_name) + " VALUES", context, database_name, comment); if (!res.out) throw Exception("LOGICAL ERROR:", ErrorCodes::LOGICAL_ERROR); @@ -186,7 +187,7 @@ static inline void dumpDataForTables( for (; iterator != master_info.need_dumping_tables.end() && !is_cancelled(); ++iterator) { const auto & table_name = iterator->first; - String comment = String("Dumping ") + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name); + String comment = "Materialize MySQL step 1: execute MySQL DDL for dump data"; tryToExecuteQuery(iterator->second, context, database_name, comment); /// create table. BlockOutputStreamPtr out = std::make_shared(master_info.version, getTableOutput(database_name, table_name, context)); @@ -395,7 +396,8 @@ void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr try { - tryToExecuteQuery(query_event.query, global_context, database_name, ""); + String comment = "Materialize MySQL step 2: execute MySQL DDL for sync data"; + tryToExecuteQuery(query_prefix + query_event.query, global_context, query_event.schema, comment); } catch (Exception & exception) { diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index 006147a6cd7..19a9f318398 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -44,6 +44,7 @@ private: mutable mysqlxx::Pool pool; mutable MySQLClient client; MaterializeMySQLSettings * settings; + String query_prefix; struct Buffers { diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index e934e7a5afc..4878fcdc9f5 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -30,20 +30,22 @@ #include #include +#include #include #include #include -#include -#include #include +#include #include #include +#include #include -#include #include #include #include +#include #include +#include #include #include #include @@ -55,17 +57,15 @@ #include #include #include +#include #include #include -#include #include #include #include #include #include -#include -#include -#include +#include #include @@ -246,12 +246,6 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & } else { - if (MaterializeMySQLSyncThread::isMySQLSyncThread()) - { - if (query->as()) - return std::make_unique(query, context); - } - throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY); } } diff --git a/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h b/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h deleted file mode 100644 index 22f71501e81..00000000000 --- a/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -namespace MySQLInterpreter -{ - -/** - */ -class InterpreterMySQLCreateQuery : public IInterpreter -{ -public: - InterpreterMySQLCreateQuery(const ASTPtr & query_ptr_, Context & context_); - - BlockIO execute() override; - -private: - ASTPtr query_ptr; - Context & context; - - ASTPtr getRewrittenQuery(); -}; - -} - -} diff --git a/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp b/src/Interpreters/MySQL/MySQLInterpreterFactory.cpp similarity index 93% rename from src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp rename to src/Interpreters/MySQL/MySQLInterpreterFactory.cpp index 3d273d32772..8bf783eadb8 100644 --- a/src/Interpreters/MySQL/InterpreterMySQLCreateQuery.cpp +++ b/src/Interpreters/MySQL/MySQLInterpreterFactory.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -13,10 +13,10 @@ #include #include #include -#include -#include #include #include +#include +#include namespace DB { @@ -31,14 +31,12 @@ namespace ErrorCodes namespace MySQLInterpreter { -InterpreterMySQLCreateQuery::InterpreterMySQLCreateQuery(const ASTPtr & query_ptr_, Context & context_) - : query_ptr(query_ptr_), context(context_) +std::unique_ptr MySQLInterpreterFactory::get(ASTPtr & query, Context & context, QueryProcessingStage::Enum) { -} + if (query->as()) + return std::make_unique(query, context); -BlockIO InterpreterMySQLCreateQuery::execute() -{ - return InterpreterCreateQuery(getRewrittenQuery(), context).execute(); + return std::unique_ptr(); } static inline NamesAndTypesList getColumnsList(ASTExpressionList * columns_define) @@ -235,10 +233,14 @@ static ASTPtr getOrderByPolicy( return order_by_expression; } -ASTPtr InterpreterMySQLCreateQuery::getRewrittenQuery() +void InterpreterCreateImpl::validate(const InterpreterCreateImpl::TQuery &, const Context &) +{ + +} + +ASTPtr InterpreterCreateImpl::getRewrittenQuery(const InterpreterCreateImpl::TQuery & create_query, const Context & context) { auto rewritten_query = std::make_shared(); - const auto & create_query = query_ptr->as(); /// This is dangerous, because the like table may not exists in ClickHouse if (create_query.like_table) @@ -252,8 +254,10 @@ ASTPtr InterpreterMySQLCreateQuery::getRewrittenQuery() NamesAndTypesList columns_name_and_type = getColumnsList(create_defines->columns); const auto & [primary_keys, unique_keys, keys, increment_columns] = getKeys(create_defines->columns, create_defines->indices, context, columns_name_and_type); + const auto & database_name = context.resolveDatabase(create_query.database); + if (primary_keys.empty()) - throw Exception("The " + backQuoteIfNeed(create_query.database) + "." + backQuoteIfNeed(create_query.table) + throw Exception("The " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(create_query.table) + " cannot be materialized, because there is no primary keys.", ErrorCodes::NOT_IMPLEMENTED); auto columns = std::make_shared(); diff --git a/src/Interpreters/MySQL/MySQLInterpreterFactory.h b/src/Interpreters/MySQL/MySQLInterpreterFactory.h new file mode 100644 index 00000000000..ccc13ebe1b0 --- /dev/null +++ b/src/Interpreters/MySQL/MySQLInterpreterFactory.h @@ -0,0 +1,59 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLInterpreter +{ + +class MySQLInterpreterFactory +{ +public: + static std::unique_ptr get( + ASTPtr & query, Context & context, QueryProcessingStage::Enum stage = QueryProcessingStage::Complete); +}; + +struct InterpreterCreateImpl +{ + using TQuery = MySQLParser::ASTCreateQuery; + + static void validate(const TQuery & query, const Context & context); + + static ASTPtr getRewrittenQuery(const TQuery & query, const Context & context); +}; + +template +class InterpreterMySQLQuery : public IInterpreter +{ +public: + InterpreterMySQLQuery(const ASTPtr & query_ptr_, Context & context_) : query_ptr(query_ptr_), context(context_) {} + + BlockIO execute() override + { + const typename InterpreterImpl::TQuery & query = query_ptr->as(); + + InterpreterImpl::validate(query, context); + ASTPtr rewritten_query = InterpreterImpl::getRewrittenQuery(query, context); + + if (rewritten_query) + return executeQuery("/* Rewritten MySQL DDL Query */ " + queryToString(rewritten_query), context, true); + + return BlockIO{}; + } + +private: + ASTPtr query_ptr; + Context & context; +}; + +using InterpreterMySQLCreateQuery = InterpreterMySQLQuery; + +} + +} diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 87b5da991a9..e5566304bac 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include @@ -35,6 +36,7 @@ #include #include #include +#include #include #include @@ -237,6 +239,7 @@ static void setQuerySpecificSettings(ASTPtr & ast, Context & context) } } +template static std::tuple executeQueryImpl( const char * begin, const char * end, @@ -258,7 +261,7 @@ static std::tuple executeQueryImpl( const Settings & settings = context.getSettingsRef(); - ParserQuery parser(end, settings.enable_debug_queries); + TParser parser(end, settings.enable_debug_queries); ASTPtr ast; const char * query_end; @@ -363,7 +366,7 @@ static std::tuple executeQueryImpl( /// reset Input callbacks if query is not INSERT SELECT context.resetInputCallbacks(); - auto interpreter = InterpreterFactory::get(ast, context, stage); + auto interpreter = TInterpreterFactory::get(ast, context, stage); std::shared_ptr quota; if (!interpreter->ignoreQuota()) @@ -854,4 +857,15 @@ void executeQuery( streams.onFinish(); } +BlockIO executeMySQLDDLQuery( + const String & query, Context & context, bool internal, QueryProcessingStage::Enum stage, bool may_have_embedded_data +) +{ + ASTPtr ast; + BlockIO streams; + std::tie(ast, streams) = executeQueryImpl( + query.data(), query.data() + query.size(), context, internal, stage, !may_have_embedded_data, nullptr); + return streams; +} + } diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index 2850bb3baf4..d3f52c6cea5 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -55,4 +55,12 @@ BlockIO executeQuery( bool allow_processors /// If can use processors pipeline ); +BlockIO executeMySQLDDLQuery( + const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out. + Context & context, /// DB, tables, data types, storage engines, functions, aggregate functions... + bool internal = false, /// If true, this query is caused by another query and thus needn't be registered in the ProcessList. + QueryProcessingStage::Enum stage = QueryProcessingStage::Complete, /// To which stage the query must be executed. + bool may_have_embedded_data = false /// If insert query may have embedded data +); + } diff --git a/src/Parsers/MySQL/ASTAlterQuery.cpp b/src/Parsers/MySQL/ASTAlterQuery.cpp new file mode 100644 index 00000000000..3ce0c62463c --- /dev/null +++ b/src/Parsers/MySQL/ASTAlterQuery.cpp @@ -0,0 +1,76 @@ +#include + +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +ASTPtr ASTAlterQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (command_list) + res->set(res->command_list, command_list->clone()); + + return res; +} + +void ASTAlterQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState & state, IAST::FormatStateStacked frame) const +{ + frame.need_parens = false; + + std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' '); + + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (settings.hilite ? hilite_none : ""); + + if (!table.empty()) + { + if (!database.empty()) + { + settings.ostr << indent_str << backQuoteIfNeed(database); + settings.ostr << "."; + } + settings.ostr << indent_str << backQuoteIfNeed(table); + } + + settings.ostr << settings.nl_or_ws; + FormatStateStacked frame_nested = frame; + frame_nested.need_parens = false; + ++frame_nested.indent; + static_cast(command_list)->formatImpl(settings, state, frame_nested); +} + +bool ParserAlterQuery::parseImpl(IParser::Pos & pos, ASTPtr & /*node*/, Expected & expected) +{ + ParserKeyword k_add("ADD"); + ParserKeyword k_alter_table("ALTER TABLE"); + + ASTPtr table; + + if (!k_alter_table.ignore(pos, expected)) + return false; + + if (!ParserCompoundIdentifier(false).parse(pos, table, expected)) + return false; + + if (k_add.ignore(pos, expected)) + { + ASTPtr declare_index; + ParserDeclareIndex p_index; + + /// TODO: add column + if (!p_index.parse(pos, declare_index, expected)) + return false; + } + return false; + +} +} + +} diff --git a/src/Parsers/MySQL/ASTAlterQuery.h b/src/Parsers/MySQL/ASTAlterQuery.h new file mode 100644 index 00000000000..6692b96ee76 --- /dev/null +++ b/src/Parsers/MySQL/ASTAlterQuery.h @@ -0,0 +1,75 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTAlterCommand : public IAST +{ +public: + enum Type + { + ADD_COLUMN, + ADD_INDEX, + }; + + /// For ADD INDEX + ASTDeclareIndex * add_index; + + /// For ADD COLUMN + ASTDeclareColumn * add_column; + +}; + +class ASTAlterCommandList : public IAST +{ +public: + std::vector commands; + + void add(const ASTPtr & command) + { + commands.push_back(command->as()); + children.push_back(command); + } + + String getID(char) const override { return "AlterCommandList"; } + + ASTPtr clone() const override; + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + +class ASTAlterQuery : public IAST +{ +public: + String database; + String table; + + ASTAlterCommandList * command_list = nullptr; + + ASTPtr clone() const override; + + String getID(char delim) const override { return "AlterQuery" + (delim + database) + delim + table; } +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + +class ParserAlterQuery : public IParserBase +{ +protected: + const char * getName() const override { return "alter query"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} + +} diff --git a/src/Parsers/MySQL/ParserMySQLQuery.cpp b/src/Parsers/MySQL/ParserMySQLQuery.cpp index 3d565224c88..7bab50fe264 100644 --- a/src/Parsers/MySQL/ParserMySQLQuery.cpp +++ b/src/Parsers/MySQL/ParserMySQLQuery.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -16,9 +15,6 @@ namespace MySQLParser bool ParserMySQLQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { - if (getThreadName() != "MySQLDBSync") - return false; - ParserDropQuery p_drop_query; ParserRenameQuery p_rename_query; ParserCreateQuery p_create_query; diff --git a/src/Parsers/MySQL/ParserMySQLQuery.h b/src/Parsers/MySQL/ParserMySQLQuery.h index aa52cd88a6e..6d467781aed 100644 --- a/src/Parsers/MySQL/ParserMySQLQuery.h +++ b/src/Parsers/MySQL/ParserMySQLQuery.h @@ -15,6 +15,9 @@ protected: const char * getName() const override { return "MySQL Query"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + +public: + ParserMySQLQuery(const char *, bool) {} }; } diff --git a/src/Parsers/ParserExternalDDLQuery.cpp b/src/Parsers/ParserExternalDDLQuery.cpp new file mode 100644 index 00000000000..e0b59913e69 --- /dev/null +++ b/src/Parsers/ParserExternalDDLQuery.cpp @@ -0,0 +1,5 @@ +// +// Created by coswde on 2020/7/6. +// + +#include "ParserExternalDDLQuery.h" diff --git a/src/Parsers/ParserExternalDDLQuery.h b/src/Parsers/ParserExternalDDLQuery.h new file mode 100644 index 00000000000..2cab5853127 --- /dev/null +++ b/src/Parsers/ParserExternalDDLQuery.h @@ -0,0 +1,6 @@ +#pragma once + +namespace DB +{ + +} diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 49897b9865d..144c309927b 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -17,7 +17,6 @@ #include #include #include -#include namespace DB @@ -39,7 +38,6 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserDropAccessEntityQuery drop_access_entity_p; ParserGrantQuery grant_p; ParserSetRoleQuery set_role_p; - MySQLParser::ParserMySQLQuery mysql_query_p; bool res = query_with_output_p.parse(pos, node, expected) || insert_p.parse(pos, node, expected) @@ -53,8 +51,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_row_policy_p.parse(pos, node, expected) || create_settings_profile_p.parse(pos, node, expected) || drop_access_entity_p.parse(pos, node, expected) - || grant_p.parse(pos, node, expected) - || mysql_query_p.parse(pos, node, expected); + || grant_p.parse(pos, node, expected); return res; } From 035760df9ded6c99e9b0c609ef85f00b27c8f97a Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 6 Jul 2020 21:57:45 +0800 Subject: [PATCH 080/374] ISSUES-4006 refactor for external ddl --- .../MySQL/MaterializeMySQLSyncThread.cpp | 20 ++--- src/Interpreters/Context.cpp | 5 ++ src/Interpreters/Context.h | 1 + .../InterpreterExternalDDLQuery.cpp | 61 +++++++++++++ .../InterpreterExternalDDLQuery.h | 21 +++++ src/Interpreters/InterpreterFactory.cpp | 12 ++- ...tory.cpp => InterpretersMySQLDDLQuery.cpp} | 87 +++++++++++++++---- .../MySQL/InterpretersMySQLDDLQuery.h | 79 +++++++++++++++++ .../MySQL/MySQLInterpreterFactory.h | 59 ------------- src/Interpreters/executeQuery.cpp | 18 +--- src/Interpreters/executeQuery.h | 8 -- src/Parsers/ASTExternalDDLQuery.h | 42 +++++++++ src/Parsers/MySQL/ParserMySQLQuery.cpp | 29 ------- src/Parsers/MySQL/ParserMySQLQuery.h | 25 ------ src/Parsers/ParserExternalDDLQuery.cpp | 60 ++++++++++++- src/Parsers/ParserExternalDDLQuery.h | 15 ++++ src/Parsers/ParserQuery.cpp | 31 ++++--- 17 files changed, 384 insertions(+), 189 deletions(-) create mode 100644 src/Interpreters/InterpreterExternalDDLQuery.cpp create mode 100644 src/Interpreters/InterpreterExternalDDLQuery.h rename src/Interpreters/MySQL/{MySQLInterpreterFactory.cpp => InterpretersMySQLDDLQuery.cpp} (82%) create mode 100644 src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h delete mode 100644 src/Interpreters/MySQL/MySQLInterpreterFactory.h create mode 100644 src/Parsers/ASTExternalDDLQuery.h delete mode 100644 src/Parsers/MySQL/ParserMySQLQuery.cpp delete mode 100644 src/Parsers/MySQL/ParserMySQLQuery.h diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index d629cab977a..9adaaa2b3eb 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -33,21 +33,17 @@ namespace ErrorCodes static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; -template static BlockIO tryToExecuteQuery(const String & query_to_execute, const Context & context_, const String & database, const String & comment) { try { Context context(context_); CurrentThread::QueryScope query_scope(context); - context.setCurrentDatabase(database); + context.unsafeSetCurrentDatabase(database); context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; context.setCurrentQueryId(""); // generate random query_id - if constexpr (execute_ddl) - return executeMySQLDDLQuery("/*" + comment + "*/ " + query_to_execute, context, true); - else - return executeQuery("/*" + comment + "*/ " + query_to_execute, context, true); + return executeQuery("/*" + comment + "*/ " + query_to_execute, context, true); } catch (...) { @@ -161,16 +157,16 @@ static inline void cleanOutdatedTables(const String & database_name, const Conte for (auto iterator = clean_database->getTablesIterator(context); iterator->isValid(); iterator->next()) { - String table_name = backQuoteIfNeed(iterator->name()); String comment = "Materialize MySQL step 1: execute MySQL DDL for dump data"; - tryToExecuteQuery("DROP TABLE " + table_name, context, database_name, comment); + String table_name = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(iterator->name()); + tryToExecuteQuery(" DROP TABLE " + table_name, context, database_name, comment); } } static inline BlockOutputStreamPtr getTableOutput(const String & database_name, const String & table_name, const Context & context) { String comment = "Materialize MySQL step 1: execute dump data"; - BlockIO res = tryToExecuteQuery("INSERT INTO " + backQuoteIfNeed(table_name) + " VALUES", context, database_name, comment); + BlockIO res = tryToExecuteQuery("INSERT INTO " + backQuoteIfNeed(table_name) + " VALUES", context, database_name, comment); if (!res.out) throw Exception("LOGICAL ERROR:", ErrorCodes::LOGICAL_ERROR); @@ -180,7 +176,7 @@ static inline BlockOutputStreamPtr getTableOutput(const String & database_name, static inline void dumpDataForTables( mysqlxx::Pool::Entry & connection, MaterializeMetadata & master_info, - const String & database_name, const String & mysql_database_name, + const String & query_prefix, const String & database_name, const String & mysql_database_name, const Context & context, const std::function & is_cancelled) { auto iterator = master_info.need_dumping_tables.begin(); @@ -188,7 +184,7 @@ static inline void dumpDataForTables( { const auto & table_name = iterator->first; String comment = "Materialize MySQL step 1: execute MySQL DDL for dump data"; - tryToExecuteQuery(iterator->second, context, database_name, comment); /// create table. + tryToExecuteQuery(query_prefix + " " + iterator->second, context, mysql_database_name, comment); /// create table. BlockOutputStreamPtr out = std::make_shared(master_info.version, getTableOutput(database_name, table_name, context)); MySQLBlockInputStream input( @@ -224,7 +220,7 @@ std::optional MaterializeMySQLSyncThread::prepareSynchroniz metadata.transaction(Position(metadata.binlog_position, metadata.binlog_file), [&]() { cleanOutdatedTables(database_name, global_context); - dumpDataForTables(connection, metadata, database_name, mysql_database_name, global_context, [this] { return isCancelled(); }); + dumpDataForTables(connection, metadata, query_prefix, database_name, mysql_database_name, global_context, [this] { return isCancelled(); }); }); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 01097a09cbb..d547cb4bb3b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1070,6 +1070,11 @@ void Context::setCurrentDatabase(const String & name) calculateAccessRights(); } +void Context::unsafeSetCurrentDatabase(const String & name) +{ + auto lock = getLock(); + current_database = name; +} void Context::setCurrentQueryId(const String & query_id) { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index dfe2d00873f..1bc3db46652 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -353,6 +353,7 @@ public: void setCurrentDatabase(const String & name); void setCurrentQueryId(const String & query_id); + void unsafeSetCurrentDatabase(const String & name); void killCurrentQuery(); diff --git a/src/Interpreters/InterpreterExternalDDLQuery.cpp b/src/Interpreters/InterpreterExternalDDLQuery.cpp new file mode 100644 index 00000000000..0f57d0a688b --- /dev/null +++ b/src/Interpreters/InterpreterExternalDDLQuery.cpp @@ -0,0 +1,61 @@ +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#include + +#include +#include +#include +#include +#include + +#ifdef USE_MYSQL +# include +# include +#endif + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +InterpreterExternalDDLQuery::InterpreterExternalDDLQuery(const ASTPtr & query_, Context & context_) + : query(query_), context(context_) +{ +} + +BlockIO InterpreterExternalDDLQuery::execute() +{ + const ASTExternalDDLQuery & external_ddl_query = query->as(); + + if (external_ddl_query.from->name == "MySQL") + { +#ifdef USE_MYSQL + const ASTs & arguments = external_ddl_query.from->arguments->children; + + if (arguments.size() != 2 || !arguments[0]->as() || !arguments[1]->as()) + throw Exception("MySQL External require two identifier arguments.", ErrorCodes::BAD_ARGUMENTS); + +#endif + if (external_ddl_query.external_ddl->as()) + return MySQLInterpreter::InterpreterMySQLDropQuery( + external_ddl_query.external_ddl, context, getIdentifierName(arguments[0]), + getIdentifierName(arguments[1])).execute(); + else if (external_ddl_query.external_ddl->as()) + return MySQLInterpreter::InterpreterMySQLRenameQuery( + external_ddl_query.external_ddl, context, getIdentifierName(arguments[0]), + getIdentifierName(arguments[1])).execute(); + else if (external_ddl_query.external_ddl->as()) + return MySQLInterpreter::InterpreterMySQLCreateQuery( + external_ddl_query.external_ddl, context, getIdentifierName(arguments[0]), + getIdentifierName(arguments[1])) .execute(); + } + + return BlockIO(); +} + +} diff --git a/src/Interpreters/InterpreterExternalDDLQuery.h b/src/Interpreters/InterpreterExternalDDLQuery.h new file mode 100644 index 00000000000..f6b29c20c70 --- /dev/null +++ b/src/Interpreters/InterpreterExternalDDLQuery.h @@ -0,0 +1,21 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class InterpreterExternalDDLQuery : public IInterpreter +{ +public: + InterpreterExternalDDLQuery(const ASTPtr & query_, Context & context_); + + BlockIO execute() override; +private: + const ASTPtr query; + Context & context; + +}; + +} diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 4878fcdc9f5..6dc1e732bcf 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -65,13 +65,15 @@ #include #include #include -#include +#include #include -#include -#include #include +#include +#include +#include + namespace ProfileEvents @@ -244,6 +246,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else { throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY); diff --git a/src/Interpreters/MySQL/MySQLInterpreterFactory.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp similarity index 82% rename from src/Interpreters/MySQL/MySQLInterpreterFactory.cpp rename to src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 8bf783eadb8..54f5bf8d97e 100644 --- a/src/Interpreters/MySQL/MySQLInterpreterFactory.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -31,14 +31,6 @@ namespace ErrorCodes namespace MySQLInterpreter { -std::unique_ptr MySQLInterpreterFactory::get(ASTPtr & query, Context & context, QueryProcessingStage::Enum) -{ - if (query->as()) - return std::make_unique(query, context); - - return std::unique_ptr(); -} - static inline NamesAndTypesList getColumnsList(ASTExpressionList * columns_define) { NamesAndTypesList columns_name_and_type; @@ -233,15 +225,8 @@ static ASTPtr getOrderByPolicy( return order_by_expression; } -void InterpreterCreateImpl::validate(const InterpreterCreateImpl::TQuery &, const Context &) +void InterpreterCreateImpl::validate(const InterpreterCreateImpl::TQuery & create_query, const Context &) { - -} - -ASTPtr InterpreterCreateImpl::getRewrittenQuery(const InterpreterCreateImpl::TQuery & create_query, const Context & context) -{ - auto rewritten_query = std::make_shared(); - /// This is dangerous, because the like table may not exists in ClickHouse if (create_query.like_table) throw Exception("Cannot convert create like statement to ClickHouse SQL", ErrorCodes::NOT_IMPLEMENTED); @@ -250,12 +235,22 @@ ASTPtr InterpreterCreateImpl::getRewrittenQuery(const InterpreterCreateImpl::TQu if (!create_defines || !create_defines->columns || create_defines->columns->children.empty()) throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); +} + +ASTPtr InterpreterCreateImpl::getRewrittenQuery( + const TQuery & create_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db) +{ + auto rewritten_query = std::make_shared(); + const auto & database_name = context.resolveDatabase(create_query.database); + + if (database_name != filter_mysql_db) + return {}; + + const auto & create_defines = create_query.columns_list->as(); NamesAndTypesList columns_name_and_type = getColumnsList(create_defines->columns); const auto & [primary_keys, unique_keys, keys, increment_columns] = getKeys(create_defines->columns, create_defines->indices, context, columns_name_and_type); - const auto & database_name = context.resolveDatabase(create_query.database); - if (primary_keys.empty()) throw Exception("The " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(create_query.table) + " cannot be materialized, because there is no primary keys.", ErrorCodes::NOT_IMPLEMENTED); @@ -281,6 +276,7 @@ ASTPtr InterpreterCreateImpl::getRewrittenQuery(const InterpreterCreateImpl::TQu storage->set(storage->engine, makeASTFunction("ReplacingMergeTree", std::make_shared(version_column_name))); + rewritten_query->database = clickhouse_db; rewritten_query->table = create_query.table; rewritten_query->if_not_exists = create_query.if_not_exists; rewritten_query->set(rewritten_query->storage, storage); @@ -289,6 +285,59 @@ ASTPtr InterpreterCreateImpl::getRewrittenQuery(const InterpreterCreateImpl::TQu return rewritten_query; } +void InterpreterDropImpl::validate(const InterpreterDropImpl::TQuery & /*query*/, const Context & /*context*/) +{ +} + +ASTPtr InterpreterDropImpl::getRewrittenQuery( + const InterpreterDropImpl::TQuery & drop_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db) +{ + const auto & database_name = context.resolveDatabase(drop_query.database); + + if (database_name != filter_mysql_db) + return {}; + + ASTPtr rewritten_query = drop_query.clone(); + rewritten_query->as()->database = clickhouse_db; + return rewritten_query; +} + +void InterpreterRenameImpl::validate(const InterpreterRenameImpl::TQuery & rename_query, const Context & /*context*/) +{ + if (rename_query.exchange) + throw Exception("Cannot execute exchange for external ddl query.", ErrorCodes::NOT_IMPLEMENTED); +} + +ASTPtr InterpreterRenameImpl::getRewrittenQuery( + const InterpreterRenameImpl::TQuery & rename_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db) +{ + ASTRenameQuery::Elements elements; + for (const auto & rename_element : rename_query.elements) + { + const auto & to_database = context.resolveDatabase(rename_element.to.database); + const auto & from_database = context.resolveDatabase(rename_element.from.database); + + if (to_database != from_database) + throw Exception("Cannot rename with other database for external ddl query.", ErrorCodes::NOT_IMPLEMENTED); + + if (from_database == filter_mysql_db) + { + elements.push_back(ASTRenameQuery::Element()); + elements.back().from.database = clickhouse_db; + elements.back().from.table = rename_element.from.table; + elements.back().to.database = clickhouse_db; + elements.back().to.table = rename_element.to.table; + } + } + + if (elements.empty()) + return ASTPtr{}; + + auto rewritten_query = std::make_shared(); + rewritten_query->elements = elements; + return rewritten_query; +} + } } diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h new file mode 100644 index 00000000000..b3aa8293b66 --- /dev/null +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h @@ -0,0 +1,79 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLInterpreter +{ + +struct InterpreterDropImpl +{ + using TQuery = ASTDropQuery; + + static void validate(const TQuery & query, const Context & context); + + static ASTPtr getRewrittenQuery(const TQuery & drop_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db); +}; + +struct InterpreterRenameImpl +{ + using TQuery = ASTRenameQuery; + + static void validate(const TQuery & query, const Context & context); + + static ASTPtr getRewrittenQuery(const TQuery & rename_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db); +}; + +struct InterpreterCreateImpl +{ + using TQuery = MySQLParser::ASTCreateQuery; + + static void validate(const TQuery & query, const Context & context); + + static ASTPtr getRewrittenQuery(const TQuery & query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db); +}; + +template +class InterpreterMySQLDDLQuery : public IInterpreter +{ +public: + InterpreterMySQLDDLQuery(const ASTPtr & query_ptr_, Context & context_, const String & clickhouse_db_, const String & mysql_db_) + : query_ptr(query_ptr_), context(context_), clickhouse_db(clickhouse_db_), mysql_db(mysql_db_) + { + } + + BlockIO execute() override + { + const typename InterpreterImpl::TQuery & query = query_ptr->as(); + + InterpreterImpl::validate(query, context); + ASTPtr rewritten_query = InterpreterImpl::getRewrittenQuery(query, context, clickhouse_db, mysql_db); + + if (rewritten_query) + return executeQuery("/* Rewritten MySQL DDL Query */ " + queryToString(rewritten_query), context, true); + + return BlockIO{}; + } + +private: + ASTPtr query_ptr; + Context & context; + const String clickhouse_db; + const String mysql_db; +}; + +using InterpreterMySQLDropQuery = InterpreterMySQLDDLQuery; +using InterpreterMySQLRenameQuery = InterpreterMySQLDDLQuery; +using InterpreterMySQLCreateQuery = InterpreterMySQLDDLQuery; + +} + +} diff --git a/src/Interpreters/MySQL/MySQLInterpreterFactory.h b/src/Interpreters/MySQL/MySQLInterpreterFactory.h deleted file mode 100644 index ccc13ebe1b0..00000000000 --- a/src/Interpreters/MySQL/MySQLInterpreterFactory.h +++ /dev/null @@ -1,59 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace MySQLInterpreter -{ - -class MySQLInterpreterFactory -{ -public: - static std::unique_ptr get( - ASTPtr & query, Context & context, QueryProcessingStage::Enum stage = QueryProcessingStage::Complete); -}; - -struct InterpreterCreateImpl -{ - using TQuery = MySQLParser::ASTCreateQuery; - - static void validate(const TQuery & query, const Context & context); - - static ASTPtr getRewrittenQuery(const TQuery & query, const Context & context); -}; - -template -class InterpreterMySQLQuery : public IInterpreter -{ -public: - InterpreterMySQLQuery(const ASTPtr & query_ptr_, Context & context_) : query_ptr(query_ptr_), context(context_) {} - - BlockIO execute() override - { - const typename InterpreterImpl::TQuery & query = query_ptr->as(); - - InterpreterImpl::validate(query, context); - ASTPtr rewritten_query = InterpreterImpl::getRewrittenQuery(query, context); - - if (rewritten_query) - return executeQuery("/* Rewritten MySQL DDL Query */ " + queryToString(rewritten_query), context, true); - - return BlockIO{}; - } - -private: - ASTPtr query_ptr; - Context & context; -}; - -using InterpreterMySQLCreateQuery = InterpreterMySQLQuery; - -} - -} diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index e5566304bac..87b5da991a9 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -24,7 +24,6 @@ #include #include #include -#include #include @@ -36,7 +35,6 @@ #include #include #include -#include #include #include @@ -239,7 +237,6 @@ static void setQuerySpecificSettings(ASTPtr & ast, Context & context) } } -template static std::tuple executeQueryImpl( const char * begin, const char * end, @@ -261,7 +258,7 @@ static std::tuple executeQueryImpl( const Settings & settings = context.getSettingsRef(); - TParser parser(end, settings.enable_debug_queries); + ParserQuery parser(end, settings.enable_debug_queries); ASTPtr ast; const char * query_end; @@ -366,7 +363,7 @@ static std::tuple executeQueryImpl( /// reset Input callbacks if query is not INSERT SELECT context.resetInputCallbacks(); - auto interpreter = TInterpreterFactory::get(ast, context, stage); + auto interpreter = InterpreterFactory::get(ast, context, stage); std::shared_ptr quota; if (!interpreter->ignoreQuota()) @@ -857,15 +854,4 @@ void executeQuery( streams.onFinish(); } -BlockIO executeMySQLDDLQuery( - const String & query, Context & context, bool internal, QueryProcessingStage::Enum stage, bool may_have_embedded_data -) -{ - ASTPtr ast; - BlockIO streams; - std::tie(ast, streams) = executeQueryImpl( - query.data(), query.data() + query.size(), context, internal, stage, !may_have_embedded_data, nullptr); - return streams; -} - } diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index d3f52c6cea5..2850bb3baf4 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -55,12 +55,4 @@ BlockIO executeQuery( bool allow_processors /// If can use processors pipeline ); -BlockIO executeMySQLDDLQuery( - const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out. - Context & context, /// DB, tables, data types, storage engines, functions, aggregate functions... - bool internal = false, /// If true, this query is caused by another query and thus needn't be registered in the ProcessList. - QueryProcessingStage::Enum stage = QueryProcessingStage::Complete, /// To which stage the query must be executed. - bool may_have_embedded_data = false /// If insert query may have embedded data -); - } diff --git a/src/Parsers/ASTExternalDDLQuery.h b/src/Parsers/ASTExternalDDLQuery.h new file mode 100644 index 00000000000..d7b3bdd932f --- /dev/null +++ b/src/Parsers/ASTExternalDDLQuery.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ASTExternalDDLQuery : public IAST +{ +public: + ASTFunction * from; + ASTPtr external_ddl; + + ASTPtr clone() const override + { + auto res = std::make_shared(*this); + res->children.clear(); + + if (from) + res->set(res->from, from->clone()); + + if (external_ddl) + { + res->external_ddl = external_ddl->clone(); + res->children.emplace_back(res->external_ddl); + } + + return res; + } + + String getID(char) const override { return "external ddl query"; } + + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked stacked) const override + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXTERNAL DDL FROM "; + from->formatImpl(settings, state, stacked); + external_ddl->formatImpl(settings, state, stacked); + } +}; + +} diff --git a/src/Parsers/MySQL/ParserMySQLQuery.cpp b/src/Parsers/MySQL/ParserMySQLQuery.cpp deleted file mode 100644 index 7bab50fe264..00000000000 --- a/src/Parsers/MySQL/ParserMySQLQuery.cpp +++ /dev/null @@ -1,29 +0,0 @@ -#include - -#include -#include -#include - -#include -#include - -namespace DB -{ - -namespace MySQLParser -{ - -bool ParserMySQLQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserDropQuery p_drop_query; - ParserRenameQuery p_rename_query; - ParserCreateQuery p_create_query; - /// TODO: alter table - - return p_create_query.parse(pos, node, expected) || p_drop_query.parse(pos, node, expected) - || p_rename_query.parse(pos, node, expected); -} - -} - -} diff --git a/src/Parsers/MySQL/ParserMySQLQuery.h b/src/Parsers/MySQL/ParserMySQLQuery.h deleted file mode 100644 index 6d467781aed..00000000000 --- a/src/Parsers/MySQL/ParserMySQLQuery.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -namespace MySQLParser -{ - -class ParserMySQLQuery : public IParserBase -{ -protected: - const char * getName() const override { return "MySQL Query"; } - - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; - -public: - ParserMySQLQuery(const char *, bool) {} -}; - -} - -} diff --git a/src/Parsers/ParserExternalDDLQuery.cpp b/src/Parsers/ParserExternalDDLQuery.cpp index e0b59913e69..4364b6bffa4 100644 --- a/src/Parsers/ParserExternalDDLQuery.cpp +++ b/src/Parsers/ParserExternalDDLQuery.cpp @@ -1,5 +1,57 @@ -// -// Created by coswde on 2020/7/6. -// +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif -#include "ParserExternalDDLQuery.h" +#include +#include +#include +#include +#include +#include + +#ifdef USE_MYSQL +# include +#endif + +namespace DB +{ + +bool ParserExternalDDLQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserFunction p_function; + ParserKeyword s_external("EXTERNAL DDL FROM"); + + ASTPtr from; + auto external_ddl_query = std::make_shared(); + + if (!s_external.ignore(pos, expected)) + return false; + + if (!p_function.parse(pos, from, expected)) + return false; + + external_ddl_query->set(external_ddl_query->from, from); + + bool res = false; + if (external_ddl_query->from->name == "MySQL") + { +#ifdef USE_MYSQL + ParserDropQuery p_drop_query; + ParserRenameQuery p_rename_query; + MySQLParser::ParserCreateQuery p_create_query; + /// TODO: alter table + + res = p_create_query.parse(pos, external_ddl_query->external_ddl, expected) + || p_drop_query.parse(pos, external_ddl_query->external_ddl, expected) + || p_rename_query.parse(pos, external_ddl_query->external_ddl, expected); + + if (external_ddl_query->external_ddl) + external_ddl_query->children.push_back(external_ddl_query->external_ddl); +#endif + } + + node = external_ddl_query; + return res; +} + +} diff --git a/src/Parsers/ParserExternalDDLQuery.h b/src/Parsers/ParserExternalDDLQuery.h index 2cab5853127..37cbcd4d88c 100644 --- a/src/Parsers/ParserExternalDDLQuery.h +++ b/src/Parsers/ParserExternalDDLQuery.h @@ -1,6 +1,21 @@ #pragma once +#include + namespace DB { +/** Query like this: + * EXTERNAL DDL FROM external_source(...) DROP|CREATE|RENAME ... + * Example: + * EXTERNAL DDL FROM MySQL(clickhouse_db, mysql_db) DROP TABLE mysql_db.name; + */ +class ParserExternalDDLQuery : public IParserBase +{ +protected: + const char * getName() const override { return "EXTERNAL DDL query"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + } diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 144c309927b..801f36afa14 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -1,22 +1,23 @@ -#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 +#include +#include +#include +#include namespace DB @@ -38,6 +39,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserDropAccessEntityQuery drop_access_entity_p; ParserGrantQuery grant_p; ParserSetRoleQuery set_role_p; + ParserExternalDDLQuery external_ddl_p; bool res = query_with_output_p.parse(pos, node, expected) || insert_p.parse(pos, node, expected) @@ -51,7 +53,8 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_row_policy_p.parse(pos, node, expected) || create_settings_profile_p.parse(pos, node, expected) || drop_access_entity_p.parse(pos, node, expected) - || grant_p.parse(pos, node, expected); + || grant_p.parse(pos, node, expected) + || external_ddl_p.parse(pos, node, expected); return res; } From e3f2919f5961c63da67944197de7344453562519 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 7 Jul 2020 00:22:11 +0800 Subject: [PATCH 081/374] ISSUES-4006 add alter query --- src/Parsers/MySQL/ASTAlterQuery.cpp | 67 ++++++++++++++++++++++------- src/Parsers/MySQL/ASTAlterQuery.h | 40 ++++++++--------- 2 files changed, 71 insertions(+), 36 deletions(-) diff --git a/src/Parsers/MySQL/ASTAlterQuery.cpp b/src/Parsers/MySQL/ASTAlterQuery.cpp index 3ce0c62463c..739f2da9f33 100644 --- a/src/Parsers/MySQL/ASTAlterQuery.cpp +++ b/src/Parsers/MySQL/ASTAlterQuery.cpp @@ -1,7 +1,10 @@ #include +#include +#include #include #include +#include #include namespace DB @@ -16,7 +19,10 @@ ASTPtr ASTAlterQuery::clone() const res->children.clear(); if (command_list) - res->set(res->command_list, command_list->clone()); + { + res->command_list = command_list->clone(); + res->children.emplace_back(res->command_list); + } return res; } @@ -43,34 +49,65 @@ void ASTAlterQuery::formatImpl(const IAST::FormatSettings & settings, IAST::Form FormatStateStacked frame_nested = frame; frame_nested.need_parens = false; ++frame_nested.indent; - static_cast(command_list)->formatImpl(settings, state, frame_nested); +// static_cast(command_list)->formatImpl(settings, state, frame_nested); } -bool ParserAlterQuery::parseImpl(IParser::Pos & pos, ASTPtr & /*node*/, Expected & expected) +bool ParserAlterCommand::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword k_add("ADD"); - ParserKeyword k_alter_table("ALTER TABLE"); + ParserKeyword k_drop("DROP"); + if (k_add.ignore(pos, expected)) + return parseAddCommand(pos, node, expected); + else if (k_drop.ignore(pos, expected)) + return parseDropCommand(pos, node, expected); + return false; +} +bool ParserAlterCommand::parseAddCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr declare_index; + ParserDeclareIndex p_index; + + /// TODO: add column + if (!p_index.parse(pos, declare_index, expected)) + return false; + + return true; +} +bool ParserAlterCommand::parseDropCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + + return false; +} + +bool ParserAlterQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ ASTPtr table; + ASTPtr command_list; - if (!k_alter_table.ignore(pos, expected)) + if (!ParserKeyword("ALTER TABLE").ignore(pos, expected)) return false; if (!ParserCompoundIdentifier(false).parse(pos, table, expected)) return false; - if (k_add.ignore(pos, expected)) - { - ASTPtr declare_index; - ParserDeclareIndex p_index; + if (!ParserList(std::make_unique(), std::make_unique(TokenType::Comma)).parse(pos, command_list, expected)) + return false; - /// TODO: add column - if (!p_index.parse(pos, declare_index, expected)) - return false; - } - return false; + auto alter_query = std::make_shared(); + + node = alter_query; + alter_query->command_list = command_list; + StorageID table_id = getTableIdentifier(table); + alter_query->table = table_id.table_name; + alter_query->database = table_id.database_name; + + if (alter_query->command_list) + alter_query->children.emplace_back(alter_query->command_list); + + return true; +} -} } } diff --git a/src/Parsers/MySQL/ASTAlterQuery.h b/src/Parsers/MySQL/ASTAlterQuery.h index 6692b96ee76..3e8d8fd6c8c 100644 --- a/src/Parsers/MySQL/ASTAlterQuery.h +++ b/src/Parsers/MySQL/ASTAlterQuery.h @@ -18,6 +18,11 @@ public: { ADD_COLUMN, ADD_INDEX, + DROP_CONSTRAINT, + DROP_COLUMN, + DROP_INDEX, + DROP_PRIMARY_KEY, + DROP_FOREIGN_KEY, }; /// For ADD INDEX @@ -28,40 +33,33 @@ public: }; -class ASTAlterCommandList : public IAST -{ -public: - std::vector commands; - - void add(const ASTPtr & command) - { - commands.push_back(command->as()); - children.push_back(command); - } - - String getID(char) const override { return "AlterCommandList"; } - - ASTPtr clone() const override; - -protected: - void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; -}; - class ASTAlterQuery : public IAST { public: String database; String table; - - ASTAlterCommandList * command_list = nullptr; + ASTPtr command_list; ASTPtr clone() const override; String getID(char delim) const override { return "AlterQuery" + (delim + database) + delim + table; } + protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; }; +class ParserAlterCommand : public IParserBase +{ +protected: + const char * getName() const override { return "alter command"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + + bool parseAddCommand(Pos & pos, ASTPtr & node, Expected & expected); + + bool parseDropCommand(Pos & pos, ASTPtr & node, Expected & expected); +}; + class ParserAlterQuery : public IParserBase { protected: From 3dda4d7dfb425d87245ca890d34cc7d60e8d2b8a Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 10 Jul 2020 15:29:10 +0800 Subject: [PATCH 082/374] ISSUES-4006 add alter command parser --- src/Parsers/MySQL/ASTAlterCommand.cpp | 317 ++++++++++++++++++++++++++ src/Parsers/MySQL/ASTAlterCommand.h | 85 +++++++ src/Parsers/MySQL/ASTAlterQuery.cpp | 28 --- src/Parsers/MySQL/ASTAlterQuery.h | 34 --- 4 files changed, 402 insertions(+), 62 deletions(-) create mode 100644 src/Parsers/MySQL/ASTAlterCommand.cpp create mode 100644 src/Parsers/MySQL/ASTAlterCommand.h diff --git a/src/Parsers/MySQL/ASTAlterCommand.cpp b/src/Parsers/MySQL/ASTAlterCommand.cpp new file mode 100644 index 00000000000..7219b3de1d1 --- /dev/null +++ b/src/Parsers/MySQL/ASTAlterCommand.cpp @@ -0,0 +1,317 @@ +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +bool ParserAlterCommand::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword k_add("ADD"); + ParserKeyword k_drop("DROP"); + ParserKeyword k_alter("ALTER"); + ParserKeyword k_rename("RENAME"); + + if (k_add.ignore(pos, expected)) + return parseAddCommand(pos, node, expected); + else if (k_drop.ignore(pos, expected)) + return parseDropCommand(pos, node, expected); + else if (k_alter.ignore(pos, expected)) + return parseAlterCommand(pos, node, expected); + else if (k_rename.ignore(pos, expected)) + return parseRenameCommand(pos, node, expected); + else + return parseOtherCommand(pos, node, expected); + + // | MODIFY [COLUMN] col_name column_definition [FIRST | AFTER col_name] + // | CHANGE [COLUMN] old_col_name new_col_name column_definition [FIRST | AFTER col_name] + + +} + +bool ParserAlterCommand::parseAddCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr declare_index; + ASTPtr additional_columns; + ParserDeclareIndex index_p; + ParserDeclareColumn column_p; + + auto alter_command = std::make_shared(); + + if (index_p.parse(pos, declare_index, expected)) + { + alter_command->type = ASTAlterCommand::ADD_INDEX; + alter_command->set(alter_command->index_decl, declare_index); + } + else + { + alter_command->type = ASTAlterCommand::ADD_COLUMN; + ParserKeyword("COLUMN").ignore(pos, expected); + + if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)) + { + ParserList columns_p(std::make_unique(), std::make_unique(TokenType::Comma)); + + if (!columns_p.parse(pos, additional_columns, expected)) + return false; + + if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + return false; + } + else + { + ASTPtr declare_column; + if (!column_p.parse(pos, declare_column, expected)) + return false; + + additional_columns = std::make_shared(); + additional_columns->children.emplace_back(declare_column); + + if (ParserKeyword("FIRST").ignore(pos, expected)) + alter_command->first = true; + else if (ParserKeyword("AFTER").ignore(pos, expected)) + { + ASTPtr after_column; + ParserIdentifier identifier_p; + if (!identifier_p.parse(pos, after_column, expected)) + return false; + + alter_command->column_name = getIdentifierName(after_column); + } + } + + alter_command->set(alter_command->additional_columns, additional_columns); + } + + node = alter_command; + return true; +} + +bool ParserAlterCommand::parseDropCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr name; + ParserIdentifier identifier_p; + + auto alter_command = std::make_shared(); + + if (ParserKeyword("PRIMARY KEY").ignore(pos, expected)) + { + alter_command->index_type = "PRIMARY_KEY"; + alter_command->type = ASTAlterCommand::DROP_INDEX; + } + else if (ParserKeyword("FOREIGN KEY").ignore(pos, expected)) + { + if (!identifier_p.parse(pos, name, expected)) + return false; + + alter_command->index_type = "FOREIGN"; + alter_command->type = ASTAlterCommand::DROP_INDEX; + alter_command->index_name = getIdentifierName(name); + } + else if (ParserKeyword("INDEX").ignore(pos, expected) || ParserKeyword("KEY").ignore(pos, expected)) + { + if (!identifier_p.parse(pos, name, expected)) + return false; + + alter_command->index_type = "KEY"; + alter_command->type = ASTAlterCommand::DROP_INDEX; + alter_command->index_name = getIdentifierName(name); + } + else if (ParserKeyword("CONSTRAINT").ignore(pos, expected) || ParserKeyword("CHECK").ignore(pos, expected)) + { + if (!identifier_p.parse(pos, name, expected)) + return false; + + alter_command->type = ASTAlterCommand::DROP_CHECK; + alter_command->constraint_name = getIdentifierName(name); + } + else + { + ParserKeyword("COLUMN").ignore(pos, expected); + + if (!identifier_p.parse(pos, name, expected)) + return false; + + alter_command->type = ASTAlterCommand::DROP_COLUMN; + alter_command->column_name = getIdentifierName(name); + } + + node = alter_command; + return true; +} + +bool ParserAlterCommand::parseAlterCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr name; + + ParserIdentifier identifier_p; + auto alter_command = std::make_shared(); + + if (ParserKeyword("INDEX").ignore(pos, expected)) + { + /// ALTER INDEX index_name {VISIBLE | INVISIBLE} + + if (!identifier_p.parse(pos, name, expected)) + return false; + + alter_command->index_visible = ParserKeyword("VISIBLE").ignore(pos, expected); + + if (!alter_command->index_visible && !ParserKeyword("INVISIBLE").ignore(pos, expected)) + return false; + + alter_command->type = ASTAlterCommand::MODIFY_INDEX_VISIBLE; + alter_command->index_name = getIdentifierName(name); + } + else if (ParserKeyword("CHECK").ignore(pos, expected) || ParserKeyword("CONSTRAINT").ignore(pos, expected)) + { + /// ALTER {CHECK | CONSTRAINT} symbol [NOT] ENFORCED + if (!identifier_p.parse(pos, name, expected)) + return false; + + alter_command->not_check_enforced = ParserKeyword("NOT").ignore(pos, expected); + + if (!ParserKeyword("ENFORCED").ignore(pos, expected)) + return false; + + alter_command->type = ASTAlterCommand::MODIFY_CHECK; + alter_command->constraint_name = getIdentifierName(name); + } + else + { + /// ALTER [COLUMN] col_name {SET DEFAULT {literal | (expr)} | DROP DEFAULT} + + ParserKeyword("COLUMN").ignore(pos, expected); + + if (!identifier_p.parse(pos, name, expected)) + return false; + + if (ParserKeyword("DROP DEFAULT").ignore(pos, expected)) + alter_command->type = ASTAlterCommand::DROP_COLUMN_DEFAULT; + else if (ParserKeyword("SET DEFAULT").ignore(pos, expected)) + { + ASTPtr default_expression; + ParserExpression expression_p; + + if (!expression_p.parse(pos, default_expression, expected)) + return false; + + alter_command->type = ASTAlterCommand::MODIFY_COLUMN_DEFAULT; + alter_command->set(alter_command->default_expression, default_expression); + } + else + return false; + + alter_command->column_name = getIdentifierName(name); + } + + node = alter_command; + return true; +} + +bool ParserAlterCommand::parseRenameCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr old_name; + ASTPtr new_name; + + ParserIdentifier identifier_p; + auto alter_command = std::make_shared(); + + if (ParserKeyword("COLUMN").ignore(pos, expected)) + { + if (!identifier_p.parse(pos, old_name, expected)) + return false; + + if (!ParserKeyword("TO").ignore(pos, expected)) + return false; + + if (!identifier_p.parse(pos, new_name, expected)) + return false; + + alter_command->type = ASTAlterCommand::RENAME_COLUMN; + alter_command->old_name = getIdentifierName(old_name); + alter_command->column_name = getIdentifierName(new_name); + } + else if (ParserKeyword("TO").ignore(pos, expected) || ParserKeyword("AS").ignore(pos, expected)) + { + if (!identifier_p.parse(pos, new_name, expected)) + return false; + + alter_command->type = ASTAlterCommand::RENAME_FOREIGN; + alter_command->index_name = getIdentifierName(new_name); + } + else if (ParserKeyword("INDEX").ignore(pos, expected) || ParserKeyword("KEY").ignore(pos, expected)) + { + if (!identifier_p.parse(pos, old_name, expected)) + return false; + + if (!ParserKeyword("TO").ignore(pos, expected)) + return false; + + if (!identifier_p.parse(pos, new_name, expected)) + return false; + + alter_command->type = ASTAlterCommand::RENAME_INDEX; + alter_command->old_name = getIdentifierName(old_name); + alter_command->index_name = getIdentifierName(new_name); + } + else + { + return false; + } + + node = alter_command; + return true; +} + +bool ParserAlterCommand::parseOtherCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + // | CONVERT TO CHARACTER SET charset_name [COLLATE collation_name] + // | [DEFAULT] CHARACTER SET [=] charset_name [COLLATE [=] collation_name] + // | LOCK [=] {DEFAULT | NONE | SHARED | EXCLUSIVE} + // | ORDER BY col_name [, col_name] ... + + if (ParserKeyword("FORCE").ignore(pos, expected)) + { + /// FORCE + } + else if (ParserKeyword("ALGORITHM").ignore(pos, expected)) + { + /// ALGORITHM [=] {DEFAULT | INSTANT | INPLACE | COPY} + } + else if (ParserKeyword("WITH").ignore(pos, expected) || ParserKeyword("WITHOUT").ignore(pos, expected)) + { + /// {WITHOUT | WITH} VALIDATION + } + else if (ParserKeyword("IMPORT").ignore(pos, expected) || ParserKeyword("DISCARD").ignore(pos, expected)) + { + /// {DISCARD | IMPORT} TABLESPACE + } + else if (ParserKeyword("ENABLE").ignore(pos, expected) || ParserKeyword("DISABLE").ignore(pos, expected)) + { + /// {DISABLE | ENABLE} KEYS + } + else + { + ASTPtr table_options; + ParserDeclareTableOptions table_options_p; + + if (!table_options_p.parse(pos, table_options, expected)) + return false; + + /// set. + } + + return false; +} + +} + +} diff --git a/src/Parsers/MySQL/ASTAlterCommand.h b/src/Parsers/MySQL/ASTAlterCommand.h new file mode 100644 index 00000000000..8cfe77ba81c --- /dev/null +++ b/src/Parsers/MySQL/ASTAlterCommand.h @@ -0,0 +1,85 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + + +namespace MySQLParser +{ + +class ASTAlterCommand : public IAST +{ +public: + enum Type + { + ADD_INDEX, + ADD_COLUMN, + + DROP_INDEX, + DROP_CHECK, + DROP_COLUMN, + DROP_COLUMN_DEFAULT, + + RENAME_INDEX, + RENAME_COLUMN, + RENAME_FOREIGN, + + MODIFY_CHECK, + MODIFY_COLUMN, + MODIFY_TABLE_OPTIONS, + MODIFY_INDEX_VISIBLE, + MODIFY_COLUMN_DEFAULT, + + ORDER_BY, + + NO_TYPE + }; + + Type type = NO_TYPE; + + /// For ADD INDEX + ASTDeclareIndex * index_decl; + + /// For modify default expression + IAST * default_expression; + + /// For ADD COLUMN + ASTExpressionList * additional_columns; + + bool first = false; + bool index_visible = false; + bool not_check_enforced = false; + + String old_name; + String index_type; + String index_name; + String column_name; + String constraint_name; +}; + +class ParserAlterCommand : public IParserBase +{ +protected: + const char * getName() const override { return "alter command"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + + bool parseAddCommand(Pos & pos, ASTPtr & node, Expected & expected); + + bool parseDropCommand(Pos & pos, ASTPtr & node, Expected & expected); + + bool parseAlterCommand(Pos & pos, ASTPtr & node, Expected & expected); + + bool parseRenameCommand(Pos & pos, ASTPtr & node, Expected & expected); + + bool parseOtherCommand(Pos & pos, ASTPtr & node, Expected & expected); +}; + +} + +} diff --git a/src/Parsers/MySQL/ASTAlterQuery.cpp b/src/Parsers/MySQL/ASTAlterQuery.cpp index 739f2da9f33..f97163130be 100644 --- a/src/Parsers/MySQL/ASTAlterQuery.cpp +++ b/src/Parsers/MySQL/ASTAlterQuery.cpp @@ -52,34 +52,6 @@ void ASTAlterQuery::formatImpl(const IAST::FormatSettings & settings, IAST::Form // static_cast(command_list)->formatImpl(settings, state, frame_nested); } -bool ParserAlterCommand::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserKeyword k_add("ADD"); - ParserKeyword k_drop("DROP"); - - if (k_add.ignore(pos, expected)) - return parseAddCommand(pos, node, expected); - else if (k_drop.ignore(pos, expected)) - return parseDropCommand(pos, node, expected); - return false; -} -bool ParserAlterCommand::parseAddCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) -{ - ASTPtr declare_index; - ParserDeclareIndex p_index; - - /// TODO: add column - if (!p_index.parse(pos, declare_index, expected)) - return false; - - return true; -} -bool ParserAlterCommand::parseDropCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) -{ - - return false; -} - bool ParserAlterQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr table; diff --git a/src/Parsers/MySQL/ASTAlterQuery.h b/src/Parsers/MySQL/ASTAlterQuery.h index 3e8d8fd6c8c..a973be85b56 100644 --- a/src/Parsers/MySQL/ASTAlterQuery.h +++ b/src/Parsers/MySQL/ASTAlterQuery.h @@ -11,28 +11,6 @@ namespace DB namespace MySQLParser { -class ASTAlterCommand : public IAST -{ -public: - enum Type - { - ADD_COLUMN, - ADD_INDEX, - DROP_CONSTRAINT, - DROP_COLUMN, - DROP_INDEX, - DROP_PRIMARY_KEY, - DROP_FOREIGN_KEY, - }; - - /// For ADD INDEX - ASTDeclareIndex * add_index; - - /// For ADD COLUMN - ASTDeclareColumn * add_column; - -}; - class ASTAlterQuery : public IAST { public: @@ -48,18 +26,6 @@ protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; }; -class ParserAlterCommand : public IParserBase -{ -protected: - const char * getName() const override { return "alter command"; } - - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; - - bool parseAddCommand(Pos & pos, ASTPtr & node, Expected & expected); - - bool parseDropCommand(Pos & pos, ASTPtr & node, Expected & expected); -}; - class ParserAlterQuery : public IParserBase { protected: From 04c2ab5ab9e47fe4550fdf58f60a4e81f2dd9898 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 13 Jul 2020 02:23:42 +0800 Subject: [PATCH 083/374] ISSUE-4006 add alter query parser --- src/Parsers/MySQL/ASTAlterCommand.cpp | 135 ++++++--- src/Parsers/MySQL/ASTAlterCommand.h | 12 +- src/Parsers/MySQL/ASTAlterQuery.cpp | 7 +- src/Parsers/MySQL/ASTAlterQuery.h | 4 +- src/Parsers/MySQL/ASTDeclareColumn.cpp | 5 +- src/Parsers/MySQL/ASTDeclareIndex.cpp | 2 +- src/Parsers/MySQL/ASTDeclareOption.cpp | 15 +- src/Parsers/MySQL/ASTDeclareOption.h | 10 +- src/Parsers/MySQL/ASTDeclarePartition.cpp | 2 +- src/Parsers/MySQL/ASTDeclareSubPartition.cpp | 2 +- src/Parsers/MySQL/ASTDeclareTableOptions.cpp | 3 +- .../tests/gtest_alter_command_parser.cpp | 258 ++++++++++++++++++ 12 files changed, 401 insertions(+), 54 deletions(-) create mode 100644 src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp diff --git a/src/Parsers/MySQL/ASTAlterCommand.cpp b/src/Parsers/MySQL/ASTAlterCommand.cpp index 7219b3de1d1..8fa734c9997 100644 --- a/src/Parsers/MySQL/ASTAlterCommand.cpp +++ b/src/Parsers/MySQL/ASTAlterCommand.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include namespace DB @@ -13,12 +14,37 @@ namespace DB namespace MySQLParser { +ASTPtr ASTAlterCommand::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (index_decl) + res->set(res->index_decl, index_decl->clone()); + + if (default_expression) + res->set(res->default_expression, default_expression->clone()); + + if (additional_columns) + res->set(res->additional_columns, additional_columns->clone()); + + if (order_by_columns) + res->set(res->order_by_columns, additional_columns->clone()); + + if (properties) + res->set(res->properties, properties->clone()); + + return res; +} + bool ParserAlterCommand::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword k_add("ADD"); ParserKeyword k_drop("DROP"); ParserKeyword k_alter("ALTER"); ParserKeyword k_rename("RENAME"); + ParserKeyword k_modify("MODIFY"); + ParserKeyword k_change("CHANGE"); if (k_add.ignore(pos, expected)) return parseAddCommand(pos, node, expected); @@ -28,13 +54,12 @@ bool ParserAlterCommand::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & return parseAlterCommand(pos, node, expected); else if (k_rename.ignore(pos, expected)) return parseRenameCommand(pos, node, expected); + else if (k_modify.ignore(pos, expected)) + return parseModifyCommand(pos, node, expected); + else if (k_change.ignore(pos, expected)) + return parseModifyCommand(pos, node, expected, true); else return parseOtherCommand(pos, node, expected); - - // | MODIFY [COLUMN] col_name column_definition [FIRST | AFTER col_name] - // | CHANGE [COLUMN] old_col_name new_col_name column_definition [FIRST | AFTER col_name] - - } bool ParserAlterCommand::parseAddCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) @@ -273,45 +298,89 @@ bool ParserAlterCommand::parseRenameCommand(IParser::Pos & pos, ASTPtr & node, E bool ParserAlterCommand::parseOtherCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) { - // | CONVERT TO CHARACTER SET charset_name [COLLATE collation_name] - // | [DEFAULT] CHARACTER SET [=] charset_name [COLLATE [=] collation_name] - // | LOCK [=] {DEFAULT | NONE | SHARED | EXCLUSIVE} - // | ORDER BY col_name [, col_name] ... + auto alter_command = std::make_shared(); - if (ParserKeyword("FORCE").ignore(pos, expected)) + if (ParserKeyword("ORDER BY").ignore(pos, expected)) { - /// FORCE - } - else if (ParserKeyword("ALGORITHM").ignore(pos, expected)) - { - /// ALGORITHM [=] {DEFAULT | INSTANT | INPLACE | COPY} - } - else if (ParserKeyword("WITH").ignore(pos, expected) || ParserKeyword("WITHOUT").ignore(pos, expected)) - { - /// {WITHOUT | WITH} VALIDATION - } - else if (ParserKeyword("IMPORT").ignore(pos, expected) || ParserKeyword("DISCARD").ignore(pos, expected)) - { - /// {DISCARD | IMPORT} TABLESPACE - } - else if (ParserKeyword("ENABLE").ignore(pos, expected) || ParserKeyword("DISABLE").ignore(pos, expected)) - { - /// {DISABLE | ENABLE} KEYS + /// ORDER BY col_name [, col_name] ... + ASTPtr columns; + ParserList columns_p(std::make_unique(), std::make_unique(TokenType::Comma)); + + if (!columns_p.parse(pos, columns, expected)) + return false; + + alter_command->type = ASTAlterCommand::ORDER_BY; + alter_command->set(alter_command->order_by_columns, columns); } else { - ASTPtr table_options; + ParserDeclareOption options_p{ + { + OptionDescribe("FORCE", "force", std::make_shared()), + OptionDescribe("ALGORITHM", "algorithm", std::make_shared()), + OptionDescribe("WITH VALIDATION", "validation", std::make_shared()), + OptionDescribe("WITHOUT VALIDATION", "validation", std::make_shared()), + OptionDescribe("IMPORT TABLESPACE", "import_tablespace", std::make_shared()), + OptionDescribe("DISCARD TABLESPACE", "import_tablespace", std::make_shared()), + OptionDescribe("ENABLE KEYS", "enable_keys", std::make_shared()), + OptionDescribe("DISABLE KEYS", "enable_keys", std::make_shared()), + /// TODO: with collate + OptionDescribe("CONVERT TO CHARACTER SET", "charset", std::make_shared()), + OptionDescribe("CHARACTER SET", "charset", std::make_shared()), + OptionDescribe("DEFAULT CHARACTER SET", "charset", std::make_shared()), + OptionDescribe("LOCK", "lock", std::make_shared()) + } + }; + + ASTPtr properties_options; ParserDeclareTableOptions table_options_p; - if (!table_options_p.parse(pos, table_options, expected)) + if (!options_p.parse(pos, properties_options, expected) && !table_options_p.parse(pos, properties_options, expected)) return false; - /// set. + alter_command->type = ASTAlterCommand::MODIFY_PROPERTIES; + alter_command->set(alter_command->properties, properties_options); } - return false; -} - + node = alter_command; + return true; +} + +bool ParserAlterCommand::parseModifyCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected, bool exists_old_column_name) +{ + ASTPtr old_column_name; + auto alter_command = std::make_shared(); + + ParserKeyword("COLUMN").ignore(pos, expected); + if (exists_old_column_name && !ParserIdentifier().parse(pos, old_column_name, expected)) + return false; + + ASTPtr additional_column; + if (!ParserDeclareColumn().parse(pos, additional_column, expected)) + return false; + + if (ParserKeyword("FIRST").ignore(pos, expected)) + alter_command->first = true; + else if (ParserKeyword("AFTER").ignore(pos, expected)) + { + ASTPtr after_column; + ParserIdentifier identifier_p; + if (!identifier_p.parse(pos, after_column, expected)) + return false; + + alter_command->column_name = getIdentifierName(after_column); + } + + node = alter_command; + alter_command->type = ASTAlterCommand::MODIFY_COLUMN; + alter_command->set(alter_command->additional_columns, std::make_shared()); + alter_command->additional_columns->children.emplace_back(additional_column); + + if (exists_old_column_name) + alter_command->old_name = getIdentifierName(old_column_name); + + return true; +} } } diff --git a/src/Parsers/MySQL/ASTAlterCommand.h b/src/Parsers/MySQL/ASTAlterCommand.h index 8cfe77ba81c..2e7fea705b1 100644 --- a/src/Parsers/MySQL/ASTAlterCommand.h +++ b/src/Parsers/MySQL/ASTAlterCommand.h @@ -31,9 +31,9 @@ public: MODIFY_CHECK, MODIFY_COLUMN, - MODIFY_TABLE_OPTIONS, MODIFY_INDEX_VISIBLE, MODIFY_COLUMN_DEFAULT, + MODIFY_PROPERTIES, ORDER_BY, @@ -50,6 +50,8 @@ public: /// For ADD COLUMN ASTExpressionList * additional_columns; + /// For ORDER BY + ASTExpressionList * order_by_columns; bool first = false; bool index_visible = false; @@ -60,6 +62,12 @@ public: String index_name; String column_name; String constraint_name; + + IAST * properties; + + ASTPtr clone() const override; + + String getID(char delim) const override { return "AlterCommand" + (delim + std::to_string(static_cast(type))); } }; class ParserAlterCommand : public IParserBase @@ -77,6 +85,8 @@ protected: bool parseRenameCommand(Pos & pos, ASTPtr & node, Expected & expected); + bool parseModifyCommand(Pos & pos, ASTPtr & node, Expected & expected, bool exists_old_column_name = false); + bool parseOtherCommand(Pos & pos, ASTPtr & node, Expected & expected); }; diff --git a/src/Parsers/MySQL/ASTAlterQuery.cpp b/src/Parsers/MySQL/ASTAlterQuery.cpp index f97163130be..da2dc178373 100644 --- a/src/Parsers/MySQL/ASTAlterQuery.cpp +++ b/src/Parsers/MySQL/ASTAlterQuery.cpp @@ -1,11 +1,12 @@ #include #include +#include #include #include #include #include -#include +#include namespace DB { @@ -27,7 +28,7 @@ ASTPtr ASTAlterQuery::clone() const return res; } -void ASTAlterQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState & state, IAST::FormatStateStacked frame) const +/*void ASTAlterQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState & state, IAST::FormatStateStacked frame) const { frame.need_parens = false; @@ -50,7 +51,7 @@ void ASTAlterQuery::formatImpl(const IAST::FormatSettings & settings, IAST::Form frame_nested.need_parens = false; ++frame_nested.indent; // static_cast(command_list)->formatImpl(settings, state, frame_nested); -} +}*/ bool ParserAlterQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { diff --git a/src/Parsers/MySQL/ASTAlterQuery.h b/src/Parsers/MySQL/ASTAlterQuery.h index a973be85b56..37f5687a456 100644 --- a/src/Parsers/MySQL/ASTAlterQuery.h +++ b/src/Parsers/MySQL/ASTAlterQuery.h @@ -22,8 +22,8 @@ public: String getID(char delim) const override { return "AlterQuery" + (delim + database) + delim + table; } -protected: - void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +/*protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;*/ }; class ParserAlterQuery : public IParserBase diff --git a/src/Parsers/MySQL/ASTDeclareColumn.cpp b/src/Parsers/MySQL/ASTDeclareColumn.cpp index b4c49b7fe0e..66f49aed565 100644 --- a/src/Parsers/MySQL/ASTDeclareColumn.cpp +++ b/src/Parsers/MySQL/ASTDeclareColumn.cpp @@ -49,8 +49,7 @@ bool ParserDeclareColumn::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte if (!ParserDataType().parse(pos, column_data_type, expected)) return false; - if (!parseColumnDeclareOptions(pos, column_options, expected)) - return false; + parseColumnDeclareOptions(pos, column_options, expected); auto declare_column = std::make_shared(); declare_column->name = getIdentifierName(column_name); @@ -68,7 +67,7 @@ bool ParserDeclareColumn::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte } bool ParserDeclareColumn::parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & node, Expected & expected) { - ParserDeclareOption p_non_generate_options{ + ParserDeclareOptions p_non_generate_options{ { OptionDescribe("ZEROFILL", "zero_fill", std::make_unique()), OptionDescribe("NULL", "is_null", std::make_unique()), diff --git a/src/Parsers/MySQL/ASTDeclareIndex.cpp b/src/Parsers/MySQL/ASTDeclareIndex.cpp index ebb269e1361..f6a1cf92857 100644 --- a/src/Parsers/MySQL/ASTDeclareIndex.cpp +++ b/src/Parsers/MySQL/ASTDeclareIndex.cpp @@ -66,7 +66,7 @@ bool ParserDeclareIndex::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & ASTPtr declare_reference; ParserIndexColumn p_expression; - ParserDeclareOption p_index_options{ + ParserDeclareOptions p_index_options{ { OptionDescribe("KEY_BLOCK_SIZE", "key_block_size", std::make_unique()), OptionDescribe("USING", "index_type", std::make_unique()), diff --git a/src/Parsers/MySQL/ASTDeclareOption.cpp b/src/Parsers/MySQL/ASTDeclareOption.cpp index 188190e0a4f..6fc84d7afec 100644 --- a/src/Parsers/MySQL/ASTDeclareOption.cpp +++ b/src/Parsers/MySQL/ASTDeclareOption.cpp @@ -11,7 +11,8 @@ namespace DB namespace MySQLParser { -bool ParserDeclareOption::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +template +bool ParserDeclareOptionImpl::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { std::unordered_map changes; std::unordered_map> usage_parsers_cached; @@ -26,7 +27,7 @@ bool ParserDeclareOption::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte return iterator->second; }; - while (true) + do { ASTPtr value; bool found{false}; @@ -57,9 +58,7 @@ bool ParserDeclareOption::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte if (!found) break; - -// ParserToken{TokenType::Comma}.ignore(pos, expected); - } + } while (recursive); if (!changes.empty()) { @@ -69,7 +68,7 @@ bool ParserDeclareOption::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte node = options_declare; } - return true; + return !changes.empty(); } ASTPtr ASTDeclareOptions::clone() const @@ -135,6 +134,10 @@ bool ParserCharsetName::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &) return false; } + +template class ParserDeclareOptionImpl; +template class ParserDeclareOptionImpl; + } } diff --git a/src/Parsers/MySQL/ASTDeclareOption.h b/src/Parsers/MySQL/ASTDeclareOption.h index fa6c25bd914..db4a40a8515 100644 --- a/src/Parsers/MySQL/ASTDeclareOption.h +++ b/src/Parsers/MySQL/ASTDeclareOption.h @@ -58,18 +58,24 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected &) override; }; -class ParserDeclareOption : public IParserBase +template +class ParserDeclareOptionImpl : public IParserBase { protected: + bool recursive = recursive_; + std::vector options_collection; const char * getName() const override { return "option declaration"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; public: - ParserDeclareOption(const std::vector & options_collection_) : options_collection(options_collection_) {} + ParserDeclareOptionImpl(const std::vector & options_collection_) : options_collection(options_collection_) {} }; +using ParserDeclareOption = ParserDeclareOptionImpl; +using ParserDeclareOptions = ParserDeclareOptionImpl; + } diff --git a/src/Parsers/MySQL/ASTDeclarePartition.cpp b/src/Parsers/MySQL/ASTDeclarePartition.cpp index 68a56109e64..289fbe5eba4 100644 --- a/src/Parsers/MySQL/ASTDeclarePartition.cpp +++ b/src/Parsers/MySQL/ASTDeclarePartition.cpp @@ -75,7 +75,7 @@ bool ParserDeclarePartition::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect } } - if (!ParserDeclareOption{ + if (!ParserDeclareOptions{ { OptionDescribe("ENGINE", "engine", std::make_shared()), OptionDescribe("STORAGE ENGINE", "engine", std::make_shared()), diff --git a/src/Parsers/MySQL/ASTDeclareSubPartition.cpp b/src/Parsers/MySQL/ASTDeclareSubPartition.cpp index 25c04779656..050f41697dd 100644 --- a/src/Parsers/MySQL/ASTDeclareSubPartition.cpp +++ b/src/Parsers/MySQL/ASTDeclareSubPartition.cpp @@ -23,7 +23,7 @@ bool ParserDeclareSubPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & e if (!p_identifier.parse(pos, logical_name, expected)) return false; - if (!ParserDeclareOption{ + if (!ParserDeclareOptions{ { OptionDescribe("ENGINE", "engine", std::make_shared()), OptionDescribe("STORAGE ENGINE", "engine", std::make_shared()), diff --git a/src/Parsers/MySQL/ASTDeclareTableOptions.cpp b/src/Parsers/MySQL/ASTDeclareTableOptions.cpp index 6d318a20f23..5ed3dad1ae9 100644 --- a/src/Parsers/MySQL/ASTDeclareTableOptions.cpp +++ b/src/Parsers/MySQL/ASTDeclareTableOptions.cpp @@ -45,6 +45,7 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override { ParserIdentifier p_identifier; + if (!p_identifier.parse(pos, node, expected)) return false; @@ -63,7 +64,7 @@ protected: bool ParserDeclareTableOptions::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { - return ParserDeclareOption{ + return ParserDeclareOptions{ { OptionDescribe("AUTO_INCREMENT", "auto_increment", std::make_shared()), OptionDescribe("AVG_ROW_LENGTH", "avg_row_length", std::make_shared()), diff --git a/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp b/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp new file mode 100644 index 00000000000..81aed23372b --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp @@ -0,0 +1,258 @@ +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +ASTPtr tryParserQuery(IParser & parser, const String & query) +{ + return parseQuery(parser, query.data(), query.data() + query.size(), "", 0, 0); +} + +TEST(ParserAlterCommand, AddAlterCommand) +{ + ParserAlterCommand alter_p; + + ASTPtr ast = tryParserQuery(alter_p, "ADD column_name INT"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::ADD_COLUMN); + EXPECT_EQ(ast->as()->additional_columns->children.size(), 1); + EXPECT_EQ(ast->as()->additional_columns->children[0]->as()->name, "column_name"); + + ast = tryParserQuery(alter_p, "ADD COLUMN column_name INT"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::ADD_COLUMN); + EXPECT_EQ(ast->as()->additional_columns->children.size(), 1); + EXPECT_EQ(ast->as()->additional_columns->children[0]->as()->name, "column_name"); + + ast = tryParserQuery(alter_p, "ADD (column_name INT)"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::ADD_COLUMN); + EXPECT_EQ(ast->as()->additional_columns->children.size(), 1); + EXPECT_EQ(ast->as()->additional_columns->children[0]->as()->name, "column_name"); + + ast = tryParserQuery(alter_p, "ADD COLUMN (column_name INT, column_name_1 INT)"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::ADD_COLUMN); + EXPECT_EQ(ast->as()->additional_columns->children.size(), 2); + EXPECT_EQ(ast->as()->additional_columns->children[0]->as()->name, "column_name"); + EXPECT_EQ(ast->as()->additional_columns->children[1]->as()->name, "column_name_1"); + + ast = tryParserQuery(alter_p, "ADD INDEX (col_01, col_02(100), col_03 DESC) KEY_BLOCK_SIZE 3"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::ADD_INDEX); + EXPECT_EQ(ast->as()->index_decl->index_columns->children.size(), 3); + EXPECT_EQ(getIdentifierName(ast->as()->index_decl->index_columns->children[0]), "col_01"); + EXPECT_EQ(ast->as()->index_decl->index_columns->children[1]->as()->name, "col_02"); + EXPECT_EQ(getIdentifierName(ast->as()->index_decl->index_columns->children[2]), "col_03"); +} + +TEST(ParserAlterCommand, DropAlterCommand) +{ + ParserAlterCommand alter_p; + + ASTPtr ast = tryParserQuery(alter_p, "DROP CHECK constraint_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::DROP_CHECK); + EXPECT_EQ(ast->as()->constraint_name, "constraint_name"); + + ast = tryParserQuery(alter_p, "DROP CONSTRAINT constraint_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::DROP_CHECK); + EXPECT_EQ(ast->as()->constraint_name, "constraint_name"); + + ast = tryParserQuery(alter_p, "DROP KEY index_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::DROP_INDEX); + EXPECT_EQ(ast->as()->index_type, "KEY"); + EXPECT_EQ(ast->as()->index_name, "index_name"); + + ast = tryParserQuery(alter_p, "DROP INDEX index_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::DROP_INDEX); + EXPECT_EQ(ast->as()->index_type, "KEY"); + EXPECT_EQ(ast->as()->index_name, "index_name"); + + ast = tryParserQuery(alter_p, "DROP PRIMARY KEY"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::DROP_INDEX); + EXPECT_EQ(ast->as()->index_type, "PRIMARY_KEY"); + + ast = tryParserQuery(alter_p, "DROP FOREIGN KEY fk_symbol"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::DROP_INDEX); + EXPECT_EQ(ast->as()->index_name, "fk_symbol"); + EXPECT_EQ(ast->as()->index_type, "FOREIGN"); + + ast = tryParserQuery(alter_p, "DROP column_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::DROP_COLUMN); + EXPECT_EQ(ast->as()->column_name, "column_name"); + + ast = tryParserQuery(alter_p, "DROP COLUMN column_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::DROP_COLUMN); + EXPECT_EQ(ast->as()->column_name, "column_name"); +} + +TEST(ParserAlterCommand, AlterAlterCommand) +{ + ParserAlterCommand alter_p; + + ASTPtr ast = tryParserQuery(alter_p, "ALTER CHECK constraint_name NOT ENFORCED"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_CHECK); + EXPECT_TRUE(ast->as()->not_check_enforced); + EXPECT_EQ(ast->as()->constraint_name, "constraint_name"); + + ast = tryParserQuery(alter_p, "ALTER CHECK constraint_name ENFORCED"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_CHECK); + EXPECT_FALSE(ast->as()->not_check_enforced); + EXPECT_EQ(ast->as()->constraint_name, "constraint_name"); + + ast = tryParserQuery(alter_p, "ALTER CONSTRAINT constraint_name NOT ENFORCED"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_CHECK); + EXPECT_TRUE(ast->as()->not_check_enforced); + EXPECT_EQ(ast->as()->constraint_name, "constraint_name"); + + ast = tryParserQuery(alter_p, "ALTER CONSTRAINT constraint_name ENFORCED"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_CHECK); + EXPECT_FALSE(ast->as()->not_check_enforced); + EXPECT_EQ(ast->as()->constraint_name, "constraint_name"); + + ast = tryParserQuery(alter_p, "ALTER INDEX index_name VISIBLE"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_INDEX_VISIBLE); + EXPECT_TRUE(ast->as()->index_visible); + EXPECT_EQ(ast->as()->index_name, "index_name"); + + ast = tryParserQuery(alter_p, "ALTER INDEX index_name INVISIBLE"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_INDEX_VISIBLE); + EXPECT_FALSE(ast->as()->index_visible); + EXPECT_EQ(ast->as()->index_name, "index_name"); + + ast = tryParserQuery(alter_p, "ALTER column_name SET DEFAULT other_column"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_COLUMN_DEFAULT); + EXPECT_EQ(ast->as()->column_name, "column_name"); + EXPECT_EQ(getIdentifierName(ast->as()->default_expression), "other_column"); + + ast = tryParserQuery(alter_p, "ALTER column_name DROP DEFAULT"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::DROP_COLUMN_DEFAULT); + EXPECT_EQ(ast->as()->column_name, "column_name"); + + ast = tryParserQuery(alter_p, "ALTER COLUMN column_name SET DEFAULT other_column"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_COLUMN_DEFAULT); + EXPECT_EQ(ast->as()->column_name, "column_name"); + EXPECT_EQ(getIdentifierName(ast->as()->default_expression), "other_column"); + + ast = tryParserQuery(alter_p, "ALTER COLUMN column_name DROP DEFAULT"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::DROP_COLUMN_DEFAULT); + EXPECT_EQ(ast->as()->column_name, "column_name"); +} + +TEST(ParserAlterCommand, RenameAlterCommand) +{ + ParserAlterCommand alter_p; + + ASTPtr ast = tryParserQuery(alter_p, "RENAME COLUMN old_column_name TO new_column_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::RENAME_COLUMN); + EXPECT_EQ(ast->as()->old_name, "old_column_name"); + EXPECT_EQ(ast->as()->column_name, "new_column_name"); + + ast = tryParserQuery(alter_p, "RENAME KEY old_index_name TO new_index_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::RENAME_INDEX); + EXPECT_EQ(ast->as()->old_name, "old_index_name"); + EXPECT_EQ(ast->as()->index_name, "new_index_name"); + + ast = tryParserQuery(alter_p, "RENAME INDEX old_index_name TO new_index_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::RENAME_INDEX); + EXPECT_EQ(ast->as()->old_name, "old_index_name"); + EXPECT_EQ(ast->as()->index_name, "new_index_name"); + + ast = tryParserQuery(alter_p, "RENAME TO new_table_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::RENAME_FOREIGN); + EXPECT_EQ(ast->as()->index_name, "new_table_name"); +} + +TEST(ParserAlterCommand, ModifyAlterCommand) +{ + ParserAlterCommand alter_p; + + ASTPtr ast = tryParserQuery(alter_p, "MODIFY column_name INT"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_COLUMN); + EXPECT_EQ(ast->as()->column_name, ""); + EXPECT_EQ(ast->as()->old_name, ""); + EXPECT_FALSE(ast->as()->first); + EXPECT_EQ(ast->as()->additional_columns->children.size(), 1); + EXPECT_EQ(ast->as()->additional_columns->children[0]->as()->name, "column_name"); + + ast = tryParserQuery(alter_p, "MODIFY column_name INT FIRST"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_COLUMN); + EXPECT_EQ(ast->as()->column_name, ""); + EXPECT_EQ(ast->as()->old_name, ""); + EXPECT_TRUE(ast->as()->first); + EXPECT_EQ(ast->as()->additional_columns->children.size(), 1); + EXPECT_EQ(ast->as()->additional_columns->children[0]->as()->name, "column_name"); + + ast = tryParserQuery(alter_p, "MODIFY column_name INT AFTER other_column_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_COLUMN); + EXPECT_EQ(ast->as()->old_name, ""); + EXPECT_FALSE(ast->as()->first); + EXPECT_EQ(ast->as()->column_name, "other_column_name"); + EXPECT_EQ(ast->as()->additional_columns->children.size(), 1); + EXPECT_EQ(ast->as()->additional_columns->children[0]->as()->name, "column_name"); + + ast = tryParserQuery(alter_p, "MODIFY COLUMN column_name INT AFTER other_column_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_COLUMN); + EXPECT_EQ(ast->as()->old_name, ""); + EXPECT_FALSE(ast->as()->first); + EXPECT_EQ(ast->as()->column_name, "other_column_name"); + EXPECT_EQ(ast->as()->additional_columns->children.size(), 1); + EXPECT_EQ(ast->as()->additional_columns->children[0]->as()->name, "column_name"); +} + +TEST(ParserAlterCommand, ChangeAlterCommand) +{ + ParserAlterCommand alter_p; + + ASTPtr ast = tryParserQuery(alter_p, "CHANGE old_column_name new_column_name INT"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_COLUMN); + EXPECT_FALSE(ast->as()->first); + EXPECT_EQ(ast->as()->column_name, ""); + EXPECT_EQ(ast->as()->old_name, "old_column_name"); + EXPECT_EQ(ast->as()->additional_columns->children.size(), 1); + EXPECT_EQ(ast->as()->additional_columns->children[0]->as()->name, "new_column_name"); + + ast = tryParserQuery(alter_p, "CHANGE old_column_name new_column_name INT FIRST"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_COLUMN); + EXPECT_TRUE(ast->as()->first); + EXPECT_EQ(ast->as()->column_name, ""); + EXPECT_EQ(ast->as()->old_name, "old_column_name"); + EXPECT_EQ(ast->as()->additional_columns->children.size(), 1); + EXPECT_EQ(ast->as()->additional_columns->children[0]->as()->name, "new_column_name"); + + ast = tryParserQuery(alter_p, "CHANGE old_column_name new_column_name INT AFTER other_column_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_COLUMN); + EXPECT_FALSE(ast->as()->first); + EXPECT_EQ(ast->as()->column_name, "other_column_name"); + EXPECT_EQ(ast->as()->old_name, "old_column_name"); + EXPECT_EQ(ast->as()->additional_columns->children.size(), 1); + EXPECT_EQ(ast->as()->additional_columns->children[0]->as()->name, "new_column_name"); + + ast = tryParserQuery(alter_p, "CHANGE COLUMN old_column_name new_column_name INT AFTER other_column_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_COLUMN); + EXPECT_FALSE(ast->as()->first); + EXPECT_EQ(ast->as()->column_name, "other_column_name"); + EXPECT_EQ(ast->as()->old_name, "old_column_name"); + EXPECT_EQ(ast->as()->additional_columns->children.size(), 1); + EXPECT_EQ(ast->as()->additional_columns->children[0]->as()->name, "new_column_name"); +} + +TEST(ParserAlterCommand, AlterOptionsCommand) +{ + ParserAlterCommand alter_p; + + ASTPtr ast = tryParserQuery(alter_p, "ALGORITHM DEFAULT"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_PROPERTIES); + EXPECT_EQ(ast->as()->properties->as()->changes.size(), 1); + EXPECT_EQ(getIdentifierName(ast->as()->properties->as()->changes["algorithm"]), "DEFAULT"); + + ast = tryParserQuery(alter_p, "AUTO_INCREMENT 1 CHECKSUM 1"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::MODIFY_PROPERTIES); + EXPECT_EQ(ast->as()->properties->as()->changes.size(), 2); + EXPECT_EQ(ast->as()->properties->as()->changes["checksum"]->as()->value.safeGet(), 1); + EXPECT_EQ(ast->as()->properties->as()->changes["auto_increment"]->as()->value.safeGet(), 1); + + EXPECT_THROW(tryParserQuery(alter_p, "ALGORITHM DEFAULT AUTO_INCREMENT 1"), Exception); +} + From 3ec80e92ede63427aadaf0dddbe461689cc8b4da Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 13 Jul 2020 02:25:53 +0800 Subject: [PATCH 084/374] ISSUES-4006 add check test case --- src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp b/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp index 81aed23372b..b9a0d8928e6 100644 --- a/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp +++ b/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp @@ -253,6 +253,7 @@ TEST(ParserAlterCommand, AlterOptionsCommand) EXPECT_EQ(ast->as()->properties->as()->changes["checksum"]->as()->value.safeGet(), 1); EXPECT_EQ(ast->as()->properties->as()->changes["auto_increment"]->as()->value.safeGet(), 1); + EXPECT_THROW(tryParserQuery(alter_p, "FORCE ALGORITHM DEFAULT"), Exception); EXPECT_THROW(tryParserQuery(alter_p, "ALGORITHM DEFAULT AUTO_INCREMENT 1"), Exception); } From a8e3e03d8b9e98a625958859510e69069b3e357d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 13 Jul 2020 12:16:28 +0800 Subject: [PATCH 085/374] ISSUES-4006 add rewrite alter query --- .../InterpreterExternalDDLQuery.cpp | 5 ++ .../MySQL/InterpretersMySQLDDLQuery.cpp | 85 +++++++++++++++++++ .../MySQL/InterpretersMySQLDDLQuery.h | 11 +++ src/Parsers/ParserExternalDDLQuery.cpp | 4 +- 4 files changed, 104 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterExternalDDLQuery.cpp b/src/Interpreters/InterpreterExternalDDLQuery.cpp index 0f57d0a688b..2e9c594812b 100644 --- a/src/Interpreters/InterpreterExternalDDLQuery.cpp +++ b/src/Interpreters/InterpreterExternalDDLQuery.cpp @@ -11,6 +11,7 @@ #include #ifdef USE_MYSQL +# include # include # include #endif @@ -49,6 +50,10 @@ BlockIO InterpreterExternalDDLQuery::execute() return MySQLInterpreter::InterpreterMySQLRenameQuery( external_ddl_query.external_ddl, context, getIdentifierName(arguments[0]), getIdentifierName(arguments[1])).execute(); + else if (external_ddl_query.external_ddl->as()) + return MySQLInterpreter::InterpreterMySQLAlterQuery( + external_ddl_query.external_ddl, context, getIdentifierName(arguments[0]), + getIdentifierName(arguments[1])) .execute(); else if (external_ddl_query.external_ddl->as()) return MySQLInterpreter::InterpreterMySQLCreateQuery( external_ddl_query.external_ddl, context, getIdentifierName(arguments[0]), diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 54f5bf8d97e..000e650317e 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -4,8 +4,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -338,6 +340,89 @@ ASTPtr InterpreterRenameImpl::getRewrittenQuery( return rewritten_query; } +void InterpreterAlterImpl::validate(const InterpreterAlterImpl::TQuery & /*query*/, const Context & /*context*/) +{ +} + +ASTPtr InterpreterAlterImpl::getRewrittenQuery( + const InterpreterAlterImpl::TQuery & alter_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db) +{ + const auto & database_name = context.resolveDatabase(alter_query.database); + + if (database_name != filter_mysql_db) + return {}; + + auto rewritten_query = std::make_shared(); + rewritten_query->database = clickhouse_db; + rewritten_query->table = alter_query.table; + rewritten_query->set(rewritten_query->command_list, std::make_shared()); + + for (const auto & command_query : alter_query.command_list->children) + { + const auto & alter_command = command_query->as(); + + if (alter_command->type == MySQLParser::ASTAlterCommand::ADD_COLUMN) + { + const auto & additional_columns = getColumnsList(alter_command->additional_columns); + + for (const auto & additional_column : InterpreterCreateQuery::formatColumns(additional_columns)->children) + { + auto rewritten_command = std::make_shared(); + rewritten_command->type = ASTAlterCommand::ADD_COLUMN; + rewritten_command->first = alter_command->first; + rewritten_command->col_decl = additional_column; + rewritten_command->column = std::make_shared(alter_command->column_name); + rewritten_query->command_list->add(rewritten_command); + } + } + else if (alter_command->type == MySQLParser::ASTAlterCommand::DROP_COLUMN) + { + auto rewritten_command = std::make_shared(); + rewritten_command->type = ASTAlterCommand::DROP_COLUMN; + rewritten_command->column = std::make_shared(alter_command->column_name); + rewritten_query->command_list->add(rewritten_command); + } + else if (alter_command->type == MySQLParser::ASTAlterCommand::RENAME_COLUMN) + { + auto rewritten_command = std::make_shared(); + rewritten_command->type = ASTAlterCommand::RENAME_COLUMN; + rewritten_command->column = std::make_shared(alter_command->old_name); + rewritten_command->rename_to = std::make_shared(alter_command->column_name); + rewritten_query->command_list->add(rewritten_command); + } + else if (alter_command->type == MySQLParser::ASTAlterCommand::MODIFY_COLUMN) + { + String new_column_name; + + { + auto rewritten_command = std::make_shared(); + rewritten_command->type = ASTAlterCommand::MODIFY_COLUMN; + rewritten_command->first = alter_command->first; + rewritten_command->column = std::make_shared(alter_command->column_name); + const auto & modify_columns = getColumnsList(alter_command->additional_columns); + + if (modify_columns.size() != 1) + throw Exception("It is a bug", ErrorCodes::LOGICAL_ERROR); + + new_column_name = modify_columns.front().name; + rewritten_command->col_decl = InterpreterCreateQuery::formatColumns(modify_columns)->children[0]; + rewritten_query->command_list->add(rewritten_command); + } + + if (!alter_command->old_name.empty()) + { + auto rewritten_command = std::make_shared(); + rewritten_command->type = ASTAlterCommand::RENAME_COLUMN; + rewritten_command->column = std::make_shared(alter_command->old_name); + rewritten_command->rename_to = std::make_shared(new_column_name); + rewritten_query->command_list->add(rewritten_command); + } + } + } + + return rewritten_query; +} + } } diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h index b3aa8293b66..d8920194ea0 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h @@ -6,6 +6,7 @@ #include #include #include +#include #include namespace DB @@ -23,6 +24,15 @@ struct InterpreterDropImpl static ASTPtr getRewrittenQuery(const TQuery & drop_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db); }; +struct InterpreterAlterImpl +{ + using TQuery = MySQLParser::ASTAlterQuery; + + static void validate(const TQuery & query, const Context & context); + + static ASTPtr getRewrittenQuery(const TQuery & alter_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db); +}; + struct InterpreterRenameImpl { using TQuery = ASTRenameQuery; @@ -71,6 +81,7 @@ private: }; using InterpreterMySQLDropQuery = InterpreterMySQLDDLQuery; +using InterpreterMySQLAlterQuery = InterpreterMySQLDDLQuery; using InterpreterMySQLRenameQuery = InterpreterMySQLDDLQuery; using InterpreterMySQLCreateQuery = InterpreterMySQLDDLQuery; diff --git a/src/Parsers/ParserExternalDDLQuery.cpp b/src/Parsers/ParserExternalDDLQuery.cpp index 4364b6bffa4..20020da22bd 100644 --- a/src/Parsers/ParserExternalDDLQuery.cpp +++ b/src/Parsers/ParserExternalDDLQuery.cpp @@ -10,6 +10,7 @@ #include #ifdef USE_MYSQL +# include # include #endif @@ -38,11 +39,12 @@ bool ParserExternalDDLQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect #ifdef USE_MYSQL ParserDropQuery p_drop_query; ParserRenameQuery p_rename_query; + MySQLParser::ParserAlterQuery p_alter_query; MySQLParser::ParserCreateQuery p_create_query; - /// TODO: alter table res = p_create_query.parse(pos, external_ddl_query->external_ddl, expected) || p_drop_query.parse(pos, external_ddl_query->external_ddl, expected) + || p_alter_query.parse(pos, external_ddl_query->external_ddl, expected) || p_rename_query.parse(pos, external_ddl_query->external_ddl, expected); if (external_ddl_query->external_ddl) From 094ce895da12c00e7bf57f8c0d9c0e7d9453517d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 13 Jul 2020 16:53:55 +0800 Subject: [PATCH 086/374] ISSUES-4006 fix test failure --- .../MySQL/MaterializeMySQLSyncThread.cpp | 2 +- .../MySQL/InterpretersMySQLDDLQuery.cpp | 64 +++++++++++++++---- .../tests/gtest_alter_command_parser.cpp | 2 +- .../MySQL/tests/gtest_alter_parser.cpp | 22 +++++++ 4 files changed, 76 insertions(+), 14 deletions(-) create mode 100644 src/Parsers/MySQL/tests/gtest_alter_parser.cpp diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 9adaaa2b3eb..2f648256514 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -92,7 +92,7 @@ MaterializeMySQLSyncThread::MaterializeMySQLSyncThread( , mysql_database_name(mysql_database_name_), pool(std::move(pool_)), client(std::move(client_)), settings(settings_) { /// TODO: 做简单的check, 失败即报错 - query_prefix = "EXTERNAL DDL FROM MySQL(" + backQuoteIfNeed(database_name) + ", " + backQuoteIfNeed(mysql_database_name) + ")"; + query_prefix = "EXTERNAL DDL FROM MySQL(" + backQuoteIfNeed(database_name) + ", " + backQuoteIfNeed(mysql_database_name) + ") "; startSynchronization(); } diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 000e650317e..383fef04baf 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -19,6 +20,7 @@ #include #include #include +#include namespace DB { @@ -363,15 +365,39 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( if (alter_command->type == MySQLParser::ASTAlterCommand::ADD_COLUMN) { - const auto & additional_columns = getColumnsList(alter_command->additional_columns); + const auto & additional_columns_name_and_type = getColumnsList(alter_command->additional_columns); + const auto & additional_columns = InterpreterCreateQuery::formatColumns(additional_columns_name_and_type); - for (const auto & additional_column : InterpreterCreateQuery::formatColumns(additional_columns)->children) + String default_after_column; + for (size_t index = 0; index < additional_columns_name_and_type.size(); ++index) { auto rewritten_command = std::make_shared(); rewritten_command->type = ASTAlterCommand::ADD_COLUMN; rewritten_command->first = alter_command->first; - rewritten_command->col_decl = additional_column; - rewritten_command->column = std::make_shared(alter_command->column_name); + rewritten_command->col_decl = additional_columns->children[index]->clone(); + + if (!alter_command->column_name.empty()) + { + rewritten_command->column = std::make_shared(alter_command->column_name); + rewritten_command->children.push_back(rewritten_command->column); + } + else + { + if (default_after_column.empty()) + { + StoragePtr storage = DatabaseCatalog::instance().getTable({clickhouse_db, alter_query.table}, context); + Block storage_header = storage->getInMemoryMetadataPtr()->getSampleBlock(); + + /// Put the sign and version columns last + default_after_column = storage_header.getByPosition(storage_header.columns() - 3).name; + } + + rewritten_command->column = std::make_shared(default_after_column); + rewritten_command->children.push_back(rewritten_command->column); + default_after_column = rewritten_command->col_decl->as()->name; + } + + rewritten_command->children.push_back(rewritten_command->col_decl); rewritten_query->command_list->add(rewritten_command); } } @@ -384,11 +410,15 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( } else if (alter_command->type == MySQLParser::ASTAlterCommand::RENAME_COLUMN) { - auto rewritten_command = std::make_shared(); - rewritten_command->type = ASTAlterCommand::RENAME_COLUMN; - rewritten_command->column = std::make_shared(alter_command->old_name); - rewritten_command->rename_to = std::make_shared(alter_command->column_name); - rewritten_query->command_list->add(rewritten_command); + if (alter_command->old_name != alter_command->column_name) + { + /// 'RENAME column_name TO column_name' is not allowed in Clickhouse + auto rewritten_command = std::make_shared(); + rewritten_command->type = ASTAlterCommand::RENAME_COLUMN; + rewritten_command->column = std::make_shared(alter_command->old_name); + rewritten_command->rename_to = std::make_shared(alter_command->column_name); + rewritten_query->command_list->add(rewritten_command); + } } else if (alter_command->type == MySQLParser::ASTAlterCommand::MODIFY_COLUMN) { @@ -398,18 +428,25 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( auto rewritten_command = std::make_shared(); rewritten_command->type = ASTAlterCommand::MODIFY_COLUMN; rewritten_command->first = alter_command->first; - rewritten_command->column = std::make_shared(alter_command->column_name); - const auto & modify_columns = getColumnsList(alter_command->additional_columns); + auto modify_columns = getColumnsList(alter_command->additional_columns); if (modify_columns.size() != 1) throw Exception("It is a bug", ErrorCodes::LOGICAL_ERROR); new_column_name = modify_columns.front().name; + modify_columns.front().name = alter_command->old_name; rewritten_command->col_decl = InterpreterCreateQuery::formatColumns(modify_columns)->children[0]; + + if (!alter_command->column_name.empty()) + { + rewritten_command->column = std::make_shared(alter_command->column_name); + rewritten_command->children.push_back(rewritten_command->column); + } + rewritten_query->command_list->add(rewritten_command); } - if (!alter_command->old_name.empty()) + if (!alter_command->old_name.empty() && alter_command->old_name != new_column_name) { auto rewritten_command = std::make_shared(); rewritten_command->type = ASTAlterCommand::RENAME_COLUMN; @@ -420,6 +457,9 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( } } + if (rewritten_query->command_list->commands.empty()) + return {}; + return rewritten_query; } diff --git a/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp b/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp index b9a0d8928e6..5534001241c 100644 --- a/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp +++ b/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp @@ -9,7 +9,7 @@ using namespace DB; using namespace DB::MySQLParser; -ASTPtr tryParserQuery(IParser & parser, const String & query) +static inline ASTPtr tryParserQuery(IParser & parser, const String & query) { return parseQuery(parser, query.data(), query.data() + query.size(), "", 0, 0); } diff --git a/src/Parsers/MySQL/tests/gtest_alter_parser.cpp b/src/Parsers/MySQL/tests/gtest_alter_parser.cpp new file mode 100644 index 00000000000..4ebbe332710 --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_alter_parser.cpp @@ -0,0 +1,22 @@ +#include + +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +static inline ASTPtr tryParserQuery(IParser & parser, const String & query) +{ + return parseQuery(parser, query.data(), query.data() + query.size(), "", 0, 0); +} + +TEST(ParserAlterQuery, AlterQuery) +{ + ParserAlterQuery alter_p; + + ASTPtr ast = tryParserQuery(alter_p, "ALTER TABLE test_table_2 ADD COLUMN (f INT, g INT)"); + EXPECT_EQ(ast->as()->command_list->children.size(), 1); + EXPECT_EQ(ast->as()->command_list->children[0]->as()->type, ASTAlterCommand::ADD_COLUMN); +} From cee16292a689bb60ec1e20cd86c164bf12f613ce Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 13 Jul 2020 17:10:37 +0800 Subject: [PATCH 087/374] ISSUES-4006 fix restart failure --- src/Databases/MySQL/MaterializeMetadata.cpp | 21 ++++++++++--------- .../MySQL/MaterializeMySQLSyncThread.cpp | 4 ++-- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 73f7b1c3870..d85b671dfd7 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -82,6 +82,7 @@ void MaterializeMetadata::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & c bool MaterializeMetadata::checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection) { + /// TODO: MySQL 5.7 Block header{ {std::make_shared(), "Log_name"}, {std::make_shared(), "File_size"}, @@ -128,11 +129,11 @@ void MaterializeMetadata::transaction(const MySQLReplication::Position & positio WriteBufferFromFile out(persistent_tmp_path, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_TRUNC | O_CREAT | O_EXCL); /// TSV format metadata file. - writeString("Version:\t1\n", out); - writeString("Binlog File:\t" + binlog_file + "\n", out); - writeString("Executed GTID:\t" + executed_gtid_set + "\n", out); - writeString("Binlog Position:\t" + toString(binlog_position) + "\n", out); - writeString("Data Version:\t" + toString(version) + "\n", out); + writeString("Version:\t1", out); + writeString("\nBinlog File:\t" + binlog_file, out); + writeString("\nExecuted GTID:\t" + executed_gtid_set, out); + writeString("\nBinlog Position:\t" + toString(binlog_position), out); + writeString("\nData Version:\t" + toString(version), out); out.next(); out.sync(); @@ -148,14 +149,14 @@ MaterializeMetadata::MaterializeMetadata(mysqlxx::PoolWithFailover::Entry & conn if (Poco::File(persistent_path).exists()) { ReadBufferFromFile in(persistent_path, DBMS_DEFAULT_BUFFER_SIZE); - assertString("Version:\t1\n", in); - assertString("Binlog File:\t", in); + assertString("Version:\t1", in); + assertString("\nBinlog File:\t", in); readString(binlog_file, in); - assertString("Executed GTID:\t", in); + assertString("\nExecuted GTID:\t", in); readString(executed_gtid_set, in); - assertString("Binlog Position:\t", in); + assertString("\nBinlog Position:\t", in); readIntText(binlog_position, in); - assertString("Data Version:\t", in); + assertString("\nData Version:\t", in); readIntText(version, in); if (checkBinlogFileExists(connection)) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 2f648256514..35cd9113b5e 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -92,6 +92,7 @@ MaterializeMySQLSyncThread::MaterializeMySQLSyncThread( , mysql_database_name(mysql_database_name_), pool(std::move(pool_)), client(std::move(client_)), settings(settings_) { /// TODO: 做简单的check, 失败即报错 + /// binlog_format = ROW binlog_row_image = FULL query_prefix = "EXTERNAL DDL FROM MySQL(" + backQuoteIfNeed(database_name) + ", " + backQuoteIfNeed(mysql_database_name) + ") "; startSynchronization(); } @@ -138,7 +139,6 @@ void MaterializeMySQLSyncThread::synchronization() } catch (...) { - /// TODO: set tryLogCurrentException(log); getDatabase(database_name).setException(std::current_exception()); } @@ -169,7 +169,7 @@ static inline BlockOutputStreamPtr getTableOutput(const String & database_name, BlockIO res = tryToExecuteQuery("INSERT INTO " + backQuoteIfNeed(table_name) + " VALUES", context, database_name, comment); if (!res.out) - throw Exception("LOGICAL ERROR:", ErrorCodes::LOGICAL_ERROR); + throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR); return res.out; } From 8f28df351459c86389419e2c56fe478580f45c76 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 13 Jul 2020 17:42:44 +0800 Subject: [PATCH 088/374] ISSUES-4006 rename locality_data to MaterializeMySQL --- src/Databases/DatabaseFactory.cpp | 14 +++++++------- src/Databases/MySQL/DatabaseMaterializeMySQL.cpp | 12 ++++++------ src/Databases/MySQL/MaterializeMySQLSettings.h | 1 - 3 files changed, 13 insertions(+), 14 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 71fdc15f70f..6017b95e123 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -98,7 +98,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String #if USE_MYSQL - else if (engine_name == "MySQL") + else if (engine_name == "MySQL" || engine_name == "MaterializeMySQL") { const ASTFunction * engine = engine_define->engine; if (!engine->arguments || engine->arguments->children.size() != 4) @@ -119,15 +119,15 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String const auto & [remote_host_name, remote_port] = parseAddress(host_name_and_port, 3306); auto mysql_pool = mysqlxx::Pool(mysql_database_name, remote_host_name, mysql_user_name, mysql_user_password, remote_port); - auto materialize_mode_settings = std::make_unique(); - - if (engine_define->settings) - materialize_mode_settings->loadFromQuery(*engine_define); - - if (materialize_mode_settings->locality_data) + if (engine_name == "MaterializeMySQL") { MySQLClient client(remote_host_name, remote_port, mysql_user_name, mysql_user_password); + auto materialize_mode_settings = std::make_unique(); + + if (engine_define->settings) + materialize_mode_settings->loadFromQuery(*engine_define); + return std::make_shared( context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_pool), std::move(client) , std::move(materialize_mode_settings)); diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 502fcc4d62b..ed246825b5c 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -121,7 +121,7 @@ time_t DatabaseMaterializeMySQL::getObjectMetadataModificationTime(const String void DatabaseMaterializeMySQL::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) { if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) - throw Exception("MySQL database in locality_data mode does not support create table.", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("MaterializeMySQL database not support create table.", ErrorCodes::NOT_IMPLEMENTED); getNestedDatabase()->createTable(context, name, table, query); } @@ -129,7 +129,7 @@ void DatabaseMaterializeMySQL::createTable(const Context & context, const String void DatabaseMaterializeMySQL::dropTable(const Context & context, const String & name, bool no_delay) { if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) - throw Exception("MySQL database in locality_data mode does not support drop table.", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("MaterializeMySQL database not support drop table.", ErrorCodes::NOT_IMPLEMENTED); getNestedDatabase()->dropTable(context, name, no_delay); } @@ -137,7 +137,7 @@ void DatabaseMaterializeMySQL::dropTable(const Context & context, const String & void DatabaseMaterializeMySQL::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) { if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) - throw Exception("MySQL database in locality_data mode does not support attach table.", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("MaterializeMySQL database not support attach table.", ErrorCodes::NOT_IMPLEMENTED); getNestedDatabase()->attachTable(name, table, relative_table_path); } @@ -145,7 +145,7 @@ void DatabaseMaterializeMySQL::attachTable(const String & name, const StoragePtr StoragePtr DatabaseMaterializeMySQL::detachTable(const String & name) { if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) - throw Exception("MySQL database in locality_data mode does not support detach table.", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("MaterializeMySQL database not support detach table.", ErrorCodes::NOT_IMPLEMENTED); return getNestedDatabase()->detachTable(name); } @@ -153,7 +153,7 @@ StoragePtr DatabaseMaterializeMySQL::detachTable(const String & name) void DatabaseMaterializeMySQL::renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange) { if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) - throw Exception("MySQL database in locality_data mode does not support rename table.", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("MaterializeMySQL database not support rename table.", ErrorCodes::NOT_IMPLEMENTED); getNestedDatabase()->renameTable(context, name, to_database, to_name, exchange); } @@ -161,7 +161,7 @@ void DatabaseMaterializeMySQL::renameTable(const Context & context, const String void DatabaseMaterializeMySQL::alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) { if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) - throw Exception("MySQL database in locality_data mode does not support alter table.", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("MaterializeMySQL database not support alter table.", ErrorCodes::NOT_IMPLEMENTED); getNestedDatabase()->alterTable(context, table_id, metadata); } diff --git a/src/Databases/MySQL/MaterializeMySQLSettings.h b/src/Databases/MySQL/MaterializeMySQLSettings.h index 77e8514d473..6f64865afac 100644 --- a/src/Databases/MySQL/MaterializeMySQLSettings.h +++ b/src/Databases/MySQL/MaterializeMySQLSettings.h @@ -14,7 +14,6 @@ class ASTStorage; struct MaterializeMySQLSettings : public SettingsCollection { #define LIST_OF_MATERIALIZE_MODE_SETTINGS(M) \ - M(SettingBool, locality_data, false, "", 0) \ M(SettingUInt64, max_rows_in_buffer, DEFAULT_BLOCK_SIZE, "", 0) \ M(SettingUInt64, max_bytes_in_buffer, DBMS_DEFAULT_BUFFER_SIZE, "", 0) \ M(SettingUInt64, max_rows_in_buffers, DEFAULT_BLOCK_SIZE, "", 0) \ From 4e161bb42a38a2e8157bd6a5d993471f8ab6ce38 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 13 Jul 2020 17:58:08 +0800 Subject: [PATCH 089/374] ISSUES-4006 remove unused metric --- src/Common/CurrentMetrics.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index afbf14b4a38..4bab9ef2844 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -13,7 +13,6 @@ M(BackgroundMovePoolTask, "Number of active tasks in BackgroundProcessingPool for moves") \ M(BackgroundSchedulePoolTask, "Number of active tasks in BackgroundSchedulePool. This pool is used for periodic ReplicatedMergeTree tasks, like cleaning old data parts, altering data parts, replica re-initialization, etc.") \ M(BackgroundBufferFlushSchedulePoolTask, "Number of active tasks in BackgroundBufferFlushSchedulePool. This pool is used for periodic Buffer flushes") \ - M(BackgroundMySQLSyncSchedulePoolTask, "Number of active tasks in BackgroundMySQLSyncSchedulePoolTask. This pool is used for MySQL Materialize Database sync.") \ M(BackgroundDistributedSchedulePoolTask, "Number of active tasks in BackgroundDistributedSchedulePool. This pool is used for distributed sends that is done in background.") \ M(CacheDictionaryUpdateQueueBatches, "Number of 'batches' (a set of keys) in update queue in CacheDictionaries.") \ M(CacheDictionaryUpdateQueueKeys, "Exact number of keys in update queue in CacheDictionaries.") \ From f5b66810ce4baf44e1c99e4f364a5189be8a4e50 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 13 Jul 2020 22:43:05 +0800 Subject: [PATCH 090/374] ISSUES-4006 add integration test --- .../__init__.py | 0 .../configs/remote_servers.xml | 12 +++ .../test_materialize_mysql_database/test.py | 76 +++++++++++++++++++ 3 files changed, 88 insertions(+) create mode 100644 tests/integration/test_materialize_mysql_database/__init__.py create mode 100644 tests/integration/test_materialize_mysql_database/configs/remote_servers.xml create mode 100644 tests/integration/test_materialize_mysql_database/test.py diff --git a/tests/integration/test_materialize_mysql_database/__init__.py b/tests/integration/test_materialize_mysql_database/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_materialize_mysql_database/configs/remote_servers.xml b/tests/integration/test_materialize_mysql_database/configs/remote_servers.xml new file mode 100644 index 00000000000..de8e5865f12 --- /dev/null +++ b/tests/integration/test_materialize_mysql_database/configs/remote_servers.xml @@ -0,0 +1,12 @@ + + + + + + node1 + 9000 + + + + + diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py new file mode 100644 index 00000000000..f61281f6f98 --- /dev/null +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -0,0 +1,76 @@ +import time +import contextlib + +import pymysql.cursors +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +clickhouse_node = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], with_mysql=True) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +class MySQLNodeInstance: + def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=3308): + self.user = user + self.port = port + self.hostname = hostname + self.password = password + self.mysql_connection = None # lazy init + + def query(self, execution_query): + if self.mysql_connection is None: + self.mysql_connection = pymysql.connect(user=self.user, password=self.password, host=self.hostname, port=self.port) + with self.mysql_connection.cursor() as cursor: + cursor.execute(execution_query) + + def close(self): + if self.mysql_connection is not None: + self.mysql_connection.close() + + +def test_clickhouse_ddl_for_materialize_mysql_database(started_cluster): + pass + +def test_mysql_ddl_for_materialize_mysql_database(started_cluster): + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") + assert 'test_database' in clickhouse_node.query('SHOW DATABASES') + + mysql_node.query('CREATE TABLE `test_database`.`test_table_1` ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE = InnoDB;') + assert 'test_table_1' in clickhouse_node.query('SHOW TABLES FROM test_database') + + mysql_node.query('CREATE TABLE `test_database`.`test_table_2` ( `id` int(11) NOT NULL PRIMARY KEY) ENGINE = InnoDB;') + assert 'test_table_2' in clickhouse_node.query('SHOW TABLES FROM test_database') + + mysql_node.query('RENAME TABLE `test_table_2` TO `test_database`.`test_table_3`') + assert 'test_table_3' in clickhouse_node.query('SHOW TABLES FROM test_database') + assert 'test_table_2' not in clickhouse_node.query('SHOW TABLES FROM test_database') + + mysql_node.query('DROP TABLE `test_database`.`test_table_3`') + assert 'test_table_3' not in clickhouse_node.query('SHOW TABLES FROM test_database') + + mysql_node.query('ALTER TABLE `test_database`.`test_table_1` ADD COLUMN `add_column` int(11) NOT NULL') + assert 'add_column' in clickhouse_node.query("DESC `test_database`.`test_table_1`"); + # + # time.sleep(3) # Because the unit of MySQL modification time is seconds, modifications made in the same second cannot be obtained + # mysql_node.query('ALTER TABLE `test_database`.`test_table` DROP COLUMN `add_column`') + # assert 'add_column' not in clickhouse_node.query("SELECT name FROM system.columns WHERE table = 'test_table' AND database = 'test_database'") + # + # mysql_node.query('DROP TABLE `test_database`.`test_table`;') + # assert 'test_table' not in clickhouse_node.query('SHOW TABLES FROM test_database') + # + # mysql_node.query("DROP DATABASE test_database") + + # TODO support + # clickhouse_node.query("DROP DATABASE test_database") + # assert 'test_database' not in clickhouse_node.query('SHOW DATABASES') \ No newline at end of file From eace0153bd3e5ee981457000192b5cec1d6cfffe Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 14 Jul 2020 11:05:17 +0800 Subject: [PATCH 091/374] ISSUES-4006 fix test failure --- src/Databases/DatabaseFactory.cpp | 3 ++- src/Databases/MySQL/DatabaseConnectionMySQL.cpp | 1 + src/Databases/MySQL/DatabaseConnectionMySQL.h | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 6017b95e123..17dc2eeb804 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -133,7 +133,8 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String , std::move(materialize_mode_settings)); } - return std::make_shared(context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_pool)); + return std::make_shared( + context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_pool)); } catch (...) { diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 3b6b0e10b4c..48c52773216 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -71,6 +71,7 @@ DatabaseConnectionMySQL::DatabaseConnectionMySQL( , mysql_pool(std::move(pool)) { empty(); /// test database is works fine. + thread = ThreadFromGlobalPool{&DatabaseConnectionMySQL::cleanOutdatedTables, this}; } bool DatabaseConnectionMySQL::empty() const diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.h b/src/Databases/MySQL/DatabaseConnectionMySQL.h index ba50facc81a..49c2c65bfa4 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.h +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.h @@ -91,7 +91,7 @@ private: void fetchLatestTablesStructureIntoCache(const std::map & tables_modification_time) const; - ThreadFromGlobalPool thread{&DatabaseConnectionMySQL::cleanOutdatedTables, this}; + ThreadFromGlobalPool thread; }; } From 1930b8f584e42da52f19faf84f1a858775fc8081 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 14 Jul 2020 22:46:51 +0800 Subject: [PATCH 092/374] ISSUES-4006 support drop materialize mysql database engine --- src/Core/MySQLClient.cpp | 29 ++----------------- src/Core/MySQLClient.h | 6 +--- src/Core/MySQLProtocol.h | 17 +++++++++++ src/Databases/DatabaseFactory.cpp | 4 +-- .../MySQL/DatabaseMaterializeMySQL.cpp | 29 ++++++++++++++++--- .../MySQL/DatabaseMaterializeMySQL.h | 2 ++ .../MySQL/MaterializeMySQLSyncThread.cpp | 26 ++++++++++------- .../MySQL/MaterializeMySQLSyncThread.h | 2 ++ 8 files changed, 68 insertions(+), 47 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 5cd71377a68..ffcbf798bb1 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -125,9 +125,6 @@ void MySQLClient::ping() void MySQLClient::startBinlogDump(UInt32 slave_id, String replicate_db, String binlog_file_name, UInt64 binlog_pos) { - if (dump_thread) - return; - /// Set binlog checksum to CRC32. String checksum = "CRC32"; writeCommand(Command::COM_QUERY, "SET @master_binlog_checksum = '" + checksum + "'"); @@ -145,34 +142,14 @@ void MySQLClient::startBinlogDump(UInt32 slave_id, String replicate_db, String b binlog_pos = binlog_pos < 4 ? 4 : binlog_pos; BinlogDump binlog_dump(binlog_pos, binlog_file_name, slave_id); packet_sender->sendPacket(binlog_dump, true); - dump_thread.emplace([this]() - { - while (true) - { - try - { - packet_sender->receivePacket(replication); - auto receive_event = replication.readOneEvent(); - events.push(std::make_pair(receive_event, replication.getPosition())); - } - catch(...) - { - tryLogCurrentException("MySQLClient"); - /// TODO: maybe sleep? - } - } - }); } BinlogEventPtr MySQLClient::readOneBinlogEvent(UInt64 milliseconds) { - std::pair event; + if (packet_sender->tryReceivePacket(replication, milliseconds)) + return replication.readOneEvent(); - if (!events.tryPop(event, milliseconds)) - return {}; - - last_position = event.second; - return event.first; + return {}; } } diff --git a/src/Core/MySQLClient.h b/src/Core/MySQLClient.h index 753e2a6cd44..ff0b1fbd3a5 100644 --- a/src/Core/MySQLClient.h +++ b/src/Core/MySQLClient.h @@ -11,7 +11,6 @@ #include #include #include -#include namespace DB @@ -37,7 +36,7 @@ public: void startBinlogDump(UInt32 slave_id, String replicate_db, String binlog_file_name, UInt64 binlog_pos); BinlogEventPtr readOneBinlogEvent(UInt64 milliseconds = 0); - Position getPosition() const { return last_position; } + Position getPosition() const { return replication.getPosition(); } private: String host; @@ -59,9 +58,6 @@ private: std::unique_ptr socket; std::optional address; std::shared_ptr packet_sender; - Position last_position; - std::optional dump_thread; - ConcurrentBoundedQueue> events{1}; void handshake(); void registerSlaveOnMaster(UInt32 slave_id); diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 1615ae7c7c6..694e6d24baf 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -407,6 +407,23 @@ public: packet.readPayload(*in, sequence_id); } + bool tryReceivePacket(ReadPacket & packet, UInt64 millisecond = 0) + { + if (millisecond != 0) + { + ReadBufferFromPocoSocket * socket_in = typeid_cast(in); + + if (!socket_in) + throw Exception("LOGICAL ERROR: Attempt to pull the duration in a non socket stream", ErrorCodes::LOGICAL_ERROR); + + if (!socket_in->poll(millisecond * 1000)) + return false; + } + + packet.readPayload(*in, sequence_id); + return true; + } + template void sendPacket(const T & packet, bool flush = false) { diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 17dc2eeb804..cbd5f055f31 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -79,11 +79,11 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String const String & engine_name = engine_define->engine->name; const UUID & uuid = create.uuid; - if (engine_name != "MySQL" && engine_name != "Lazy" && engine_define->engine->arguments) + if (engine_name != "MySQL" && engine_name != "MaterializeMySQL" && engine_name != "Lazy" && engine_define->engine->arguments) throw Exception("Database engine " + engine_name + " cannot have arguments", ErrorCodes::BAD_ARGUMENTS); if (engine_define->engine->parameters || engine_define->partition_by || engine_define->primary_key || engine_define->order_by || - engine_define->sample_by || (engine_name != "MySQL" && engine_define->settings)) + engine_define->sample_by || (engine_name != "MaterializeMySQL" && engine_define->settings)) throw Exception("Database engine " + engine_name + " cannot have parameters, primary_key, order_by, sample_by, settings", ErrorCodes::UNKNOWN_ELEMENT_IN_AST); diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index ed246825b5c..ed99e84749d 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -19,7 +20,7 @@ namespace ErrorCodes DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( const Context & context, const String & database_name_, const String & metadata_path_, const IAST * database_engine_define_ , const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, std::unique_ptr settings_) - : IDatabase(database_name_), engine_define(database_engine_define_->clone()) + : IDatabase(database_name_), global_context(context.getGlobalContext()), engine_define(database_engine_define_->clone()) , nested_database(std::make_shared(database_name_, metadata_path_, context)) , settings(std::move(settings_)), log(&Poco::Logger::get("DatabaseMaterializeMySQL")) , materialize_thread(context, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), settings.get()) @@ -75,7 +76,13 @@ void DatabaseMaterializeMySQL::loadStoredObjects(Context & context, bool has_for void DatabaseMaterializeMySQL::shutdown() { - getNestedDatabase()->shutdown(); + materialize_thread.stopSynchronization(); + + auto iterator = nested_database->getTablesIterator(global_context, {}); + + /// We only shutdown the table, The tables is cleaned up when destructed database + for (; iterator->isValid(); iterator->next()) + iterator->table()->shutdown(); } bool DatabaseMaterializeMySQL::empty() const @@ -168,12 +175,26 @@ void DatabaseMaterializeMySQL::alterTable(const Context & context, const Storage bool DatabaseMaterializeMySQL::shouldBeEmptyOnDetach() const { - return getNestedDatabase()->shouldBeEmptyOnDetach(); + return false; } void DatabaseMaterializeMySQL::drop(const Context & context) { - getNestedDatabase()->drop(context); + DatabasePtr nested_database = getNestedDatabase(); + + if (nested_database->shouldBeEmptyOnDetach()) + { + for (auto iterator = nested_database->getTablesIterator(context, {}); iterator->isValid(); iterator->next()) + { + TableExclusiveLockHolder table_lock = iterator->table()->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + nested_database->dropTable(context, iterator->name(), true); + } + + /// Remove metadata info + Poco::File(getMetadataPath() + "/.metadata").remove(false); + } + + nested_database->drop(context); } bool DatabaseMaterializeMySQL::isTableExist(const String & name, const Context & context) const diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h index a30390a4628..78f8e293224 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -23,6 +23,8 @@ public: void setException(const std::exception_ptr & exception); protected: + const Context & global_context; + ASTPtr engine_define; DatabasePtr nested_database; std::unique_ptr settings; diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 35cd9113b5e..e0baf98e252 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -68,16 +68,7 @@ MaterializeMySQLSyncThread::~MaterializeMySQLSyncThread() { try { - if (!sync_quit) - { - { - sync_quit = true; - std::lock_guard lock(sync_mutex); - } - - sync_cond.notify_one(); - /// TODO: join thread - } + stopSynchronization(); } catch (...) { @@ -144,6 +135,20 @@ void MaterializeMySQLSyncThread::synchronization() } } +void MaterializeMySQLSyncThread::stopSynchronization() +{ + if (!sync_quit) + { + { + sync_quit = true; + std::lock_guard lock(sync_mutex); + } + + sync_cond.notify_one(); + background_thread_pool->join(); + } +} + void MaterializeMySQLSyncThread::startSynchronization() { /// TODO: reset exception. @@ -404,6 +409,7 @@ void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr } } } + bool MaterializeMySQLSyncThread::isMySQLSyncThread() { return getThreadName() == MYSQL_BACKGROUND_THREAD_NAME; diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index 19a9f318398..76297a55f54 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -30,6 +30,8 @@ public: const Context & context, const String & database_name_, const String & mysql_database_name_ , mysqlxx::Pool && pool_, MySQLClient && client_, MaterializeMySQLSettings * settings_); + void stopSynchronization(); + void startSynchronization(); static bool isMySQLSyncThread(); From 94a2a7791cc4910d42dc8bc8634705489b9a00b8 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 15 Jul 2020 13:26:37 +0800 Subject: [PATCH 093/374] ISSUES-4006 add some integration test --- .../MySQL/InterpretersMySQLDDLQuery.cpp | 16 ++ .../test_materialize_mysql_database/test.py | 182 +++++++++++++++--- 2 files changed, 171 insertions(+), 27 deletions(-) diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 383fef04baf..879f430b661 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -376,6 +376,22 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( rewritten_command->first = alter_command->first; rewritten_command->col_decl = additional_columns->children[index]->clone(); + const auto & column_declare = alter_command->additional_columns->children[index]->as(); + if (column_declare && column_declare->column_options) + { + /// We need to add default expression for fill data + const auto & column_options = column_declare->column_options->as(); + + const auto & default_expression_it = column_options->changes.find("default"); + if (default_expression_it != column_options->changes.end()) + { + ASTColumnDeclaration * col_decl = rewritten_command->col_decl->as(); + col_decl->default_specifier = "DEFAULT"; + col_decl->default_expression = default_expression_it->second->clone(); + col_decl->children.emplace_back(col_decl->default_expression); + } + } + if (!alter_command->column_name.empty()) { rewritten_command->column = std::make_shared(alter_command->column_name); diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index f61281f6f98..b2ee00c3455 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -36,41 +36,169 @@ class MySQLNodeInstance: self.mysql_connection.close() -def test_clickhouse_ddl_for_materialize_mysql_database(started_cluster): - pass +def check_query(query, result_set, retry_count=3, interval_seconds=1): + for index in range(retry_count): + if result_set == clickhouse_node.query(query): + return + time.sleep(interval_seconds) + raise Exception("") -def test_mysql_ddl_for_materialize_mysql_database(started_cluster): + +def test_mysql_drop_table_for_materialize_mysql_database(started_cluster): + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + + mysql_node.query("DROP TABLE test_database.test_table_1;") + + mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + + mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") + + # create mapping + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") + + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") + check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") + + mysql_node.query("INSERT INTO test_database.test_table2 VALUES(1)(2)(3)(4)(5)(6)") + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + + mysql_node.query("DROP TABLE test_database.test_table_1;") + mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") + check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") + + +def test_mysql_create_table_for_materialize_mysql_database(started_cluster): + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + # existed before the mapping was created + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + # it already has some data + mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1)(2)(3)(5)(6)(7)") + + # create mapping + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") + + # Check for pre-existing status + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query("SELECT * FROM test_database.test_table_1 ORDER BY id FORMAT TSV", "1\n2\n3\n5\n6\n7") + + mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1)(2)(3)(4)(5)(6)") + check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6") + + +def test_mysql_rename_table_for_materialize_mysql_database(started_cluster): + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + + mysql_node.query("RENAME TABLE test_database.test_table_1 TO test_database.test_table_2") + + # create mapping + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") + + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") + mysql_node.query("RENAME TABLE test_database.test_table_2 TO test_database.test_table_1") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + + +def test_mysql_alter_add_column_for_materialize_mysql_database(started_cluster): + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + + mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_1 INT NOT NULL") + mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") + mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") + mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT id") + + # create mapping + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") + + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query("DESC test_database.test_table_1 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t") + mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t") + mysql_node.query("ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") + mysql_node.query("ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT id") + check_query("DESC test_database.test_table_2 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t") + + +def test_mysql_alter_drop_column_for_materialize_mysql_database(started_cluster): + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") + + mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column") + + # create mapping + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") + + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query("DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t") + mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\ndrop_column\tInt32\t\t\t\t") + mysql_node.query("ALTER TABLE test_database.test_table_2 DROP COLUMN drop_column") + check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t") + + +def test_mysql_alter_rename_column_for_materialize_mysql_database(started_cluster): with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + # maybe should test rename primary key? + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") + + mysql_node.query("ALTER TABLE test_database.test_table_1 RENAME COLUMN rename_column TO new_column_name") + + # create mapping clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") - assert 'test_database' in clickhouse_node.query('SHOW DATABASES') - mysql_node.query('CREATE TABLE `test_database`.`test_table_1` ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE = InnoDB;') - assert 'test_table_1' in clickhouse_node.query('SHOW TABLES FROM test_database') + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query("DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t") + mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nrename_column\tInt32\t\t\t\t") + mysql_node.query("ALTER TABLE test_database.test_table_2 RENAME COLUMN rename_column TO new_column_name") + check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t") - mysql_node.query('CREATE TABLE `test_database`.`test_table_2` ( `id` int(11) NOT NULL PRIMARY KEY) ENGINE = InnoDB;') - assert 'test_table_2' in clickhouse_node.query('SHOW TABLES FROM test_database') - mysql_node.query('RENAME TABLE `test_table_2` TO `test_database`.`test_table_3`') - assert 'test_table_3' in clickhouse_node.query('SHOW TABLES FROM test_database') - assert 'test_table_2' not in clickhouse_node.query('SHOW TABLES FROM test_database') +def test_mysql_alter_modify_column_for_materialize_mysql_database(started_cluster): + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query('DROP TABLE `test_database`.`test_table_3`') - assert 'test_table_3' not in clickhouse_node.query('SHOW TABLES FROM test_database') + # maybe should test rename primary key? + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") - mysql_node.query('ALTER TABLE `test_database`.`test_table_1` ADD COLUMN `add_column` int(11) NOT NULL') - assert 'add_column' in clickhouse_node.query("DESC `test_database`.`test_table_1`"); - # - # time.sleep(3) # Because the unit of MySQL modification time is seconds, modifications made in the same second cannot be obtained - # mysql_node.query('ALTER TABLE `test_database`.`test_table` DROP COLUMN `add_column`') - # assert 'add_column' not in clickhouse_node.query("SELECT name FROM system.columns WHERE table = 'test_table' AND database = 'test_database'") - # - # mysql_node.query('DROP TABLE `test_database`.`test_table`;') - # assert 'test_table' not in clickhouse_node.query('SHOW TABLES FROM test_database') - # - # mysql_node.query("DROP DATABASE test_database") + mysql_node.query("ALTER TABLE test_database.test_table_1 MODIFY COLUMN modify_column INT") - # TODO support - # clickhouse_node.query("DROP DATABASE test_database") - # assert 'test_database' not in clickhouse_node.query('SHOW DATABASES') \ No newline at end of file + # create mapping + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") + + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query("DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t") + mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tInt32\t\t\t\t") + mysql_node.query("ALTER TABLE test_database.test_table_1 MODIFY COLUMN modify_column INT") + check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t") + + +# TODO: need support ALTER TABLE table_name ADD COLUMN column_name, RENAME COLUMN column_name TO new_column_name; +# def test_mysql_alter_change_column_for_materialize_mysql_database(started_cluster): +# pass From 1c17878f310e9ccd578d8cf35255417f6a995004 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 15 Jul 2020 13:30:25 +0800 Subject: [PATCH 094/374] ISSUES-4006 try fix build failure --- .../MySQL/DatabaseMaterializeTablesIterator.h | 10 +++++----- src/Databases/MySQL/MaterializeMySQLSyncThread.cpp | 12 ++++-------- 2 files changed, 9 insertions(+), 13 deletions(-) diff --git a/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h b/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h index 16fd86fc59a..bc46e4da46e 100644 --- a/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h +++ b/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h @@ -9,19 +9,19 @@ namespace DB class DatabaseMaterializeTablesIterator final : public IDatabaseTablesIterator { public: - virtual void next() { nested_iterator->next(); } + void next() override { nested_iterator->next(); } - virtual bool isValid() const { return nested_iterator->isValid(); } + bool isValid() const override { return nested_iterator->isValid(); } - virtual const String & name() const { return nested_iterator->name(); } + const String & name() const override { return nested_iterator->name(); } - virtual const StoragePtr & table() const + const StoragePtr & table() const override { StoragePtr storage = std::make_shared(nested_iterator->table()); return tables.emplace_back(storage); } - virtual UUID uuid() const { return nested_iterator->uuid(); } + UUID uuid() const override { return nested_iterator->uuid(); } DatabaseMaterializeTablesIterator(DatabaseTablesIteratorPtr nested_iterator_) : nested_iterator(std::move(nested_iterator_)) {} diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index e0baf98e252..b6e63e8f412 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -465,15 +465,11 @@ MaterializeMySQLSyncThread::Buffers::BufferAndSortingColumnsPtr MaterializeMySQL BufferAndSortingColumnsPtr & buffer_and_soring_columns = data.try_emplace( table_name, std::make_shared(metadata.getSampleBlockNonMaterialized(), std::vector{})).first->second; - if (StorageMergeTree * table_merge_tree = storage->as()) - { - Names required_for_sorting_key = metadata.getColumnsRequiredForSortingKey(); + Names required_for_sorting_key = metadata.getColumnsRequiredForSortingKey(); - for (const auto & required_name_for_sorting_key : required_for_sorting_key) - buffer_and_soring_columns->second.emplace_back( - buffer_and_soring_columns->first.getPositionByName(required_name_for_sorting_key)); - - } + for (const auto & required_name_for_sorting_key : required_for_sorting_key) + buffer_and_soring_columns->second.emplace_back( + buffer_and_soring_columns->first.getPositionByName(required_name_for_sorting_key)); return buffer_and_soring_columns; } From e471f805b56e094229e17ad1c51bce7f0b7f885d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 15 Jul 2020 15:03:09 +0800 Subject: [PATCH 095/374] ISSUES-4006 try fix build failure & update integration test --- .../MySQL/DatabaseMaterializeMySQL.cpp | 45 ++++++++++++------- .../MySQL/InterpretersMySQLDDLQuery.cpp | 25 +++++++---- .../test_materialize_mysql_database/test.py | 8 ++-- 3 files changed, 48 insertions(+), 30 deletions(-) diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index ed99e84749d..62b6c41dbb8 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -1,13 +1,19 @@ -#include +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif -#include -#include -#include -#include -#include -#include -#include -#include +#if USE_MYSQL + +# include + +# include +# include +# include +# include +# include +# include +# include +# include namespace DB { @@ -43,9 +49,9 @@ DatabasePtr DatabaseMaterializeMySQL::getNestedDatabase() const { std::rethrow_exception(exception); } - catch (Exception & exception) + catch (Exception & ex) { - throw Exception(exception); + throw Exception(ex); } } @@ -180,21 +186,24 @@ bool DatabaseMaterializeMySQL::shouldBeEmptyOnDetach() const void DatabaseMaterializeMySQL::drop(const Context & context) { - DatabasePtr nested_database = getNestedDatabase(); + DatabasePtr database = getNestedDatabase(); - if (nested_database->shouldBeEmptyOnDetach()) + if (database->shouldBeEmptyOnDetach()) { - for (auto iterator = nested_database->getTablesIterator(context, {}); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(context, {}); iterator->isValid(); iterator->next()) { TableExclusiveLockHolder table_lock = iterator->table()->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - nested_database->dropTable(context, iterator->name(), true); + database->dropTable(context, iterator->name(), true); } /// Remove metadata info - Poco::File(getMetadataPath() + "/.metadata").remove(false); + Poco::File metadata(getMetadataPath() + "/.metadata"); + + if (metadata.exists()) + metadata.remove(false); } - nested_database->drop(context); + database->drop(context); } bool DatabaseMaterializeMySQL::isTableExist(const String & name, const Context & context) const @@ -222,3 +231,5 @@ DatabaseTablesIteratorPtr DatabaseMaterializeMySQL::getTablesIterator(const Cont } } + +#endif diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 879f430b661..8558445f82f 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -392,22 +392,29 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( } } + if (default_after_column.empty()) + { + StoragePtr storage = DatabaseCatalog::instance().getTable({clickhouse_db, alter_query.table}, context); + Block storage_header = storage->getInMemoryMetadataPtr()->getSampleBlock(); + + /// Put the sign and version columns last + default_after_column = storage_header.getByPosition(storage_header.columns() - 3).name; + } + if (!alter_command->column_name.empty()) { rewritten_command->column = std::make_shared(alter_command->column_name); rewritten_command->children.push_back(rewritten_command->column); + + /// For example(when add_column_1 is last column): + /// ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT AFTER add_column_1, ADD COLUMN add_column_4 INT + /// In this case, we still need to change the default after column + + if (alter_command->column_name == default_after_column) + default_after_column = rewritten_command->col_decl->as()->name; } else { - if (default_after_column.empty()) - { - StoragePtr storage = DatabaseCatalog::instance().getTable({clickhouse_db, alter_query.table}, context); - Block storage_header = storage->getInMemoryMetadataPtr()->getSampleBlock(); - - /// Put the sign and version columns last - default_after_column = storage_header.getByPosition(storage_header.columns() - 3).name; - } - rewritten_command->column = std::make_shared(default_after_column); rewritten_command->children.push_back(rewritten_command->column); default_after_column = rewritten_command->col_decl->as()->name; diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index b2ee00c3455..c5ad719bcd1 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -119,20 +119,20 @@ def test_mysql_alter_add_column_for_materialize_mysql_database(started_cluster): mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_1 INT NOT NULL") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT id") + mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT (id)") # create mapping clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query("DESC test_database.test_table_1 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t") + check_query("DESC test_database.test_table_1 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\nadd_column_4\tInt32\t\t\t\t") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t") mysql_node.query("ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") - mysql_node.query("ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT id") - check_query("DESC test_database.test_table_2 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t") + mysql_node.query("ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT (id)") + check_query("DESC test_database.test_table_2 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\nadd_column_4\tInt32\t\t\t\t") def test_mysql_alter_drop_column_for_materialize_mysql_database(started_cluster): From 95b8afb8eb0a8c48e5e0ae8c97716f60a063e41b Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 16 Jul 2020 11:34:28 +0800 Subject: [PATCH 096/374] ISSUES-4006 try fix unit test failure --- src/Core/MySQLClient.cpp | 2 +- src/Databases/MySQL/MaterializeMetadata.cpp | 8 +++---- src/Databases/MySQL/MaterializeMetadata.h | 2 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 23 +++++++++++++------ .../MySQL/InterpretersMySQLDDLQuery.cpp | 12 +++++----- src/Parsers/MySQL/ASTDeclareOption.cpp | 6 +++-- src/Parsers/MySQL/ASTDeclareOption.h | 4 +--- src/Parsers/MySQL/ASTDeclarePartition.cpp | 13 +++++++---- src/Parsers/MySQL/ASTDeclareSubPartition.cpp | 13 +++++++---- .../MySQL/tests/gtest_constraint_parser.cpp | 10 ++++---- 10 files changed, 56 insertions(+), 37 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index ffcbf798bb1..06c26f200e4 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -18,7 +18,7 @@ MySQLClient::MySQLClient(const String & host_, UInt16 port_, const String & user } MySQLClient::MySQLClient(MySQLClient && other) - : host(other.host), port(other.port), user(other.user), password(other.password) + : host(std::move(other.host)), port(other.port), user(std::move(other.user)), password(std::move(other.password)) , client_capability_flags(other.client_capability_flags) { } diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index d85b671dfd7..99b515ebea8 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -17,7 +17,7 @@ static std::unordered_map fetchTablesCreateQuery( const mysqlxx::PoolWithFailover::Entry & connection, const String & database_name, const std::vector & fetch_tables) { std::unordered_map tables_create_query; - for (size_t index = 0; index < fetch_tables.size(); ++index) + for (const auto & fetch_table_name : fetch_tables) { Block show_create_table_header{ {std::make_shared(), "Table"}, @@ -25,14 +25,14 @@ static std::unordered_map fetchTablesCreateQuery( }; MySQLBlockInputStream show_create_table( - connection, "SHOW CREATE TABLE " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(fetch_tables[index]), + connection, "SHOW CREATE TABLE " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(fetch_table_name), show_create_table_header, DEFAULT_BLOCK_SIZE); Block create_query_block = show_create_table.read(); if (!create_query_block || create_query_block.rows() != 1) throw Exception("LOGICAL ERROR mysql show create return more rows.", ErrorCodes::LOGICAL_ERROR); - tables_create_query[fetch_tables[index]] = create_query_block.getByName("Create Table").column->getDataAt(0).toString(); + tables_create_query[fetch_table_name] = create_query_block.getByName("Create Table").column->getDataAt(0).toString(); } return tables_create_query; @@ -80,7 +80,7 @@ void MaterializeMetadata::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & c executed_gtid_set = (*master_status.getByPosition(4).column)[0].safeGet(); } -bool MaterializeMetadata::checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection) +bool MaterializeMetadata::checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection) const { /// TODO: MySQL 5.7 Block header{ diff --git a/src/Databases/MySQL/MaterializeMetadata.h b/src/Databases/MySQL/MaterializeMetadata.h index 2020674c86c..be6f83f8166 100644 --- a/src/Databases/MySQL/MaterializeMetadata.h +++ b/src/Databases/MySQL/MaterializeMetadata.h @@ -23,7 +23,7 @@ struct MaterializeMetadata void fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & connection); - bool checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection); + bool checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection) const; void transaction(const MySQLReplication::Position & position, const std::function & fun); diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index b6e63e8f412..aa0adb3b5a0 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -6,21 +6,22 @@ #include +# include # include -# include -# include -# include # include -# include -# include # include -# include +# include +# include # include +# include # include # include # include # include # include +# include +# include +# include namespace DB { @@ -199,6 +200,14 @@ static inline void dumpDataForTables( } } +static inline UInt32 randomNumber() +{ + std::mt19937 rng; + rng.seed(std::random_device()()); + std::uniform_int_distribution dist6(std::numeric_limits::min(), std::numeric_limits::max()); + return dist6(rng); +} + std::optional MaterializeMySQLSyncThread::prepareSynchronized() { std::unique_lock lock(sync_mutex); @@ -233,7 +242,7 @@ std::optional MaterializeMySQLSyncThread::prepareSynchroniz connection->query("COMMIT").execute(); client.connect(); - client.startBinlogDump(std::rand(), mysql_database_name, metadata.binlog_file, metadata.binlog_position); + client.startBinlogDump(randomNumber(), mysql_database_name, metadata.binlog_file, metadata.binlog_position); return metadata; } catch (...) diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 8558445f82f..527738d143f 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -38,9 +38,9 @@ namespace MySQLInterpreter static inline NamesAndTypesList getColumnsList(ASTExpressionList * columns_define) { NamesAndTypesList columns_name_and_type; - for (size_t index = 0; index < columns_define->children.size(); ++index) + for (const auto & declare_column_ast : columns_define->children) { - const auto & declare_column = columns_define->children[index]->as(); + const auto & declare_column = declare_column_ast->as(); if (!declare_column || !declare_column->data_type) throw Exception("Missing type in definition of column.", ErrorCodes::UNKNOWN_TYPE); @@ -85,9 +85,9 @@ static inline std::tuplechildren.empty()) { - for (size_t index = 0; index < indices_define->children.size(); ++index) + for (const auto & declare_index_ast : indices_define->children) { - const auto & declare_index = indices_define->children[index]->as(); + const auto & declare_index = declare_index_ast->as(); /// flatten if (startsWith(declare_index->index_type, "KEY_")) @@ -102,9 +102,9 @@ static inline std::tuplechildren.size(); ++index) + for (const auto & declare_column_ast : columns_define->children) { - const auto & declare_column = columns_define->children[index]->as(); + const auto & declare_column = declare_column_ast->as(); if (declare_column->column_options) { diff --git a/src/Parsers/MySQL/ASTDeclareOption.cpp b/src/Parsers/MySQL/ASTDeclareOption.cpp index 6fc84d7afec..e8adea8a8d2 100644 --- a/src/Parsers/MySQL/ASTDeclareOption.cpp +++ b/src/Parsers/MySQL/ASTDeclareOption.cpp @@ -11,8 +11,8 @@ namespace DB namespace MySQLParser { -template -bool ParserDeclareOptionImpl::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +template +bool ParserDeclareOptionImpl::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { std::unordered_map changes; std::unordered_map> usage_parsers_cached; @@ -39,6 +39,7 @@ bool ParserDeclareOptionImpl::parseImpl(Pos & pos, ASTPtr & node, Ex { found = true; changes.insert(std::make_pair(option_describe.option_name, value)); + break; } } else if (get_parser_from_cache(option_describe.usage_name)->ignore(pos, expected)) @@ -53,6 +54,7 @@ bool ParserDeclareOptionImpl::parseImpl(Pos & pos, ASTPtr & node, Ex throw Exception("Duplicate options declare", ErrorCodes::)*/ found = true; changes.insert(std::make_pair(option_describe.option_name, value)); + break; } } diff --git a/src/Parsers/MySQL/ASTDeclareOption.h b/src/Parsers/MySQL/ASTDeclareOption.h index db4a40a8515..fa22698175a 100644 --- a/src/Parsers/MySQL/ASTDeclareOption.h +++ b/src/Parsers/MySQL/ASTDeclareOption.h @@ -58,12 +58,10 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected &) override; }; -template +template class ParserDeclareOptionImpl : public IParserBase { protected: - bool recursive = recursive_; - std::vector options_collection; const char * getName() const override { return "option declaration"; } diff --git a/src/Parsers/MySQL/ASTDeclarePartition.cpp b/src/Parsers/MySQL/ASTDeclarePartition.cpp index 289fbe5eba4..8e1d27778b5 100644 --- a/src/Parsers/MySQL/ASTDeclarePartition.cpp +++ b/src/Parsers/MySQL/ASTDeclarePartition.cpp @@ -75,7 +75,7 @@ bool ParserDeclarePartition::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect } } - if (!ParserDeclareOptions{ + ParserDeclareOptions options_p{ { OptionDescribe("ENGINE", "engine", std::make_shared()), OptionDescribe("STORAGE ENGINE", "engine", std::make_shared()), @@ -86,8 +86,10 @@ bool ParserDeclarePartition::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect OptionDescribe("MIN_ROWS", "min_rows", std::make_shared()), OptionDescribe("TABLESPACE", "tablespace", std::make_shared()), } - }.parse(pos, options, expected)) - return false; + }; + + /// Optional options + options_p.parse(pos, options, expected); ASTPtr subpartitions; if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)) @@ -107,8 +109,11 @@ bool ParserDeclarePartition::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect partition_declare->subpartitions = subpartitions; partition_declare->partition_name = partition_name->as()->name; - if (partition_declare->options) + if (options) + { + partition_declare->options = options; partition_declare->children.emplace_back(partition_declare->options); + } if (partition_declare->less_than) partition_declare->children.emplace_back(partition_declare->less_than); diff --git a/src/Parsers/MySQL/ASTDeclareSubPartition.cpp b/src/Parsers/MySQL/ASTDeclareSubPartition.cpp index 050f41697dd..1b2d9c081e6 100644 --- a/src/Parsers/MySQL/ASTDeclareSubPartition.cpp +++ b/src/Parsers/MySQL/ASTDeclareSubPartition.cpp @@ -23,7 +23,7 @@ bool ParserDeclareSubPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & e if (!p_identifier.parse(pos, logical_name, expected)) return false; - if (!ParserDeclareOptions{ + ParserDeclareOptions options_p{ { OptionDescribe("ENGINE", "engine", std::make_shared()), OptionDescribe("STORAGE ENGINE", "engine", std::make_shared()), @@ -34,15 +34,20 @@ bool ParserDeclareSubPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & e OptionDescribe("MIN_ROWS", "min_rows", std::make_shared()), OptionDescribe("TABLESPACE", "tablespace", std::make_shared()), } - }.parse(pos, options, expected)) - return false; + }; + + /// Optional options + options_p.parse(pos, options, expected); auto subpartition_declare = std::make_shared(); subpartition_declare->options = options; subpartition_declare->logical_name = logical_name->as()->name; - if (subpartition_declare->options) + if (options) + { + subpartition_declare->options = options; subpartition_declare->children.emplace_back(subpartition_declare->options); + } node = subpartition_declare; return true; diff --git a/src/Parsers/MySQL/tests/gtest_constraint_parser.cpp b/src/Parsers/MySQL/tests/gtest_constraint_parser.cpp index 7f0e9a8060d..de885bf36c8 100644 --- a/src/Parsers/MySQL/tests/gtest_constraint_parser.cpp +++ b/src/Parsers/MySQL/tests/gtest_constraint_parser.cpp @@ -16,7 +16,7 @@ TEST(ParserConstraint, CheckConstraint) String constraint_01 = "CONSTRAINT symbol_name CHECK col_01 = 1"; ASTPtr ast_constraint_01 = parseQuery(p_constraint, constraint_01.data(), constraint_01.data() + constraint_01.size(), "", 0, 0); EXPECT_EQ(ast_constraint_01->as()->constraint_name, "symbol_name"); - auto check_expression_01 = ast_constraint_01->as()->check_expression->as(); + auto * check_expression_01 = ast_constraint_01->as()->check_expression->as(); EXPECT_EQ(check_expression_01->name, "equals"); EXPECT_EQ(check_expression_01->arguments->children[0]->as()->name, "col_01"); EXPECT_EQ(check_expression_01->arguments->children[1]->as()->value.safeGet(), 1); @@ -24,7 +24,7 @@ TEST(ParserConstraint, CheckConstraint) String constraint_02 = "CONSTRAINT CHECK col_01 = 1"; ASTPtr ast_constraint_02 = parseQuery(p_constraint, constraint_02.data(), constraint_02.data() + constraint_02.size(), "", 0, 0); EXPECT_EQ(ast_constraint_02->as()->constraint_name, ""); - auto check_expression_02 = ast_constraint_02->as()->check_expression->as(); + auto * check_expression_02 = ast_constraint_02->as()->check_expression->as(); EXPECT_EQ(check_expression_02->name, "equals"); EXPECT_EQ(check_expression_02->arguments->children[0]->as()->name, "col_01"); EXPECT_EQ(check_expression_02->arguments->children[1]->as()->value.safeGet(), 1); @@ -32,7 +32,7 @@ TEST(ParserConstraint, CheckConstraint) String constraint_03 = "CHECK col_01 = 1"; ASTPtr ast_constraint_03 = parseQuery(p_constraint, constraint_03.data(), constraint_03.data() + constraint_03.size(), "", 0, 0); EXPECT_EQ(ast_constraint_03->as()->constraint_name, ""); - auto check_expression_03 = ast_constraint_03->as()->check_expression->as(); + auto * check_expression_03 = ast_constraint_03->as()->check_expression->as(); EXPECT_EQ(check_expression_03->name, "equals"); EXPECT_EQ(check_expression_03->arguments->children[0]->as()->name, "col_01"); EXPECT_EQ(check_expression_03->arguments->children[1]->as()->value.safeGet(), 1); @@ -41,7 +41,7 @@ TEST(ParserConstraint, CheckConstraint) ASTPtr ast_constraint_04 = parseQuery(p_constraint, constraint_04.data(), constraint_04.data() + constraint_04.size(), "", 0, 0); EXPECT_TRUE(ast_constraint_04->as()->enforced); EXPECT_EQ(ast_constraint_04->as()->constraint_name, ""); - auto check_expression_04 = ast_constraint_04->as()->check_expression->as(); + auto * check_expression_04 = ast_constraint_04->as()->check_expression->as(); EXPECT_EQ(check_expression_04->name, "equals"); EXPECT_EQ(check_expression_04->arguments->children[0]->as()->name, "col_01"); EXPECT_EQ(check_expression_04->arguments->children[1]->as()->value.safeGet(), 1); @@ -50,7 +50,7 @@ TEST(ParserConstraint, CheckConstraint) ASTPtr ast_constraint_05 = parseQuery(p_constraint, constraint_05.data(), constraint_05.data() + constraint_05.size(), "", 0, 0); EXPECT_FALSE(ast_constraint_05->as()->enforced); EXPECT_EQ(ast_constraint_05->as()->constraint_name, ""); - auto check_expression_05 = ast_constraint_05->as()->check_expression->as(); + auto * check_expression_05 = ast_constraint_05->as()->check_expression->as(); EXPECT_EQ(check_expression_05->name, "equals"); EXPECT_EQ(check_expression_05->arguments->children[0]->as()->name, "col_01"); EXPECT_EQ(check_expression_05->arguments->children[1]->as()->value.safeGet(), 1); From 74f354d840d2a29b67db3de05093df81fc3b7b43 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 17 Jul 2020 14:39:17 +0800 Subject: [PATCH 097/374] ISSUES-4006 add some check before create materialize mysql database --- src/Common/ErrorCodes.cpp | 1 + src/Core/MySQLReplication.cpp | 34 ++++----- src/Databases/MySQL/MaterializeMetadata.cpp | 24 ++++-- src/Databases/MySQL/MaterializeMetadata.h | 6 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 76 ++++++++++++++++--- .../MySQL/MaterializeMySQLSyncThread.h | 6 +- .../composes/mysql_5_7_compose.yml | 0 7 files changed, 106 insertions(+), 41 deletions(-) create mode 100644 tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index d63b9f9ccab..0f7d9099314 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -501,6 +501,7 @@ namespace ErrorCodes extern const int NO_RESERVATIONS_PROVIDED = 534; extern const int UNKNOWN_RAID_TYPE = 535; extern const int CANNOT_RESTORE_FROM_FIELD_DUMP = 536; + extern const int ILLEGAL_MYSQL_VARIABLE = 537; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index d31c4c0b128..6dd507ec409 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB @@ -398,7 +399,7 @@ namespace MySQLReplication case MYSQL_TYPE_TIMESTAMP: { UInt32 val = 0; payload.readStrict(reinterpret_cast(&val), 4); - row.push_back(Field{UInt64{val}}); + row.push_back(Field{val}); break; } case MYSQL_TYPE_TIME: { @@ -420,15 +421,10 @@ namespace MySQLReplication UInt32 i24 = 0; payload.readStrict(reinterpret_cast(&i24), 3); - String time_buff; - time_buff.resize(10); - sprintf( - time_buff.data(), - "%04d-%02d-%02d", - static_cast((i24 >> 9) & 0x7fff), - static_cast((i24 >> 5) & 0xf), - static_cast(i24 & 0x1f)); - row.push_back(Field{String{time_buff}}); + DayNum date_day_number = DateLUT::instance().makeDayNum( + static_cast((i24 >> 9) & 0x7fff), static_cast((i24 >> 5) & 0xf), static_cast(i24 & 0x1f)); + + row.push_back(Field(date_day_number.toUnderType())); break; } case MYSQL_TYPE_YEAR: { @@ -486,24 +482,20 @@ namespace MySQLReplication readBigEndianStrict(payload, reinterpret_cast(&val), 5); readTimeFractionalPart(payload, reinterpret_cast(&fsp), meta); - struct tm timeinfo; UInt32 year_month = readBits(val, 1, 17, 40); - timeinfo.tm_year = (year_month / 13) - 1900; - timeinfo.tm_mon = (year_month % 13) - 1; - timeinfo.tm_mday = readBits(val, 18, 5, 40); - timeinfo.tm_hour = readBits(val, 23, 5, 40); - timeinfo.tm_min = readBits(val, 28, 6, 40); - timeinfo.tm_sec = readBits(val, 34, 6, 40); + time_t date_time = DateLUT::instance().makeDateTime( + year_month / 13, year_month % 13, readBits(val, 18, 5, 40) + , readBits(val, 23, 5, 40), readBits(val, 28, 6, 40), readBits(val, 34, 6, 40) + ); - time_t time = mktime(&timeinfo); - row.push_back(Field{UInt64{static_cast(time)}}); + row.push_back(Field{UInt32(date_time)}); break; } case MYSQL_TYPE_TIMESTAMP2: { - UInt64 sec = 0, fsp = 0; + UInt32 sec = 0, fsp = 0; readBigEndianStrict(payload, reinterpret_cast(&sec), 4); readTimeFractionalPart(payload, reinterpret_cast(&fsp), meta); - row.push_back(Field{UInt64{sec}}); + row.push_back(Field{sec}); break; } case MYSQL_TYPE_NEWDECIMAL: { diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 99b515ebea8..1879a1fb79d 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -80,16 +80,26 @@ void MaterializeMetadata::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & c executed_gtid_set = (*master_status.getByPosition(4).column)[0].safeGet(); } -bool MaterializeMetadata::checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection) const +static Block getShowMasterLogHeader(const String & mysql_version) { - /// TODO: MySQL 5.7 - Block header{ + if (startsWith(mysql_version, "5.")) + { + return Block { + {std::make_shared(), "Log_name"}, + {std::make_shared(), "File_size"} + }; + } + + return Block { {std::make_shared(), "Log_name"}, {std::make_shared(), "File_size"}, {std::make_shared(), "Encrypted"} }; +} - MySQLBlockInputStream input(connection, "SHOW MASTER LOGS", header, DEFAULT_BLOCK_SIZE); +bool MaterializeMetadata::checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection, const String & mysql_version) const +{ + MySQLBlockInputStream input(connection, "SHOW MASTER LOGS", getShowMasterLogHeader(mysql_version), DEFAULT_BLOCK_SIZE); while (Block block = input.read()) { @@ -143,7 +153,9 @@ void MaterializeMetadata::transaction(const MySQLReplication::Position & positio commitMetadata(fun, persistent_tmp_path, persistent_path); } -MaterializeMetadata::MaterializeMetadata(mysqlxx::PoolWithFailover::Entry & connection, const String & path_, const String & database, bool & opened_transaction) +MaterializeMetadata::MaterializeMetadata( + mysqlxx::PoolWithFailover::Entry & connection, const String & path_, + const String & database, bool & opened_transaction, const String & mysql_version) : persistent_path(path_) { if (Poco::File(persistent_path).exists()) @@ -159,7 +171,7 @@ MaterializeMetadata::MaterializeMetadata(mysqlxx::PoolWithFailover::Entry & conn assertString("\nData Version:\t", in); readIntText(version, in); - if (checkBinlogFileExists(connection)) + if (checkBinlogFileExists(connection, mysql_version)) return; } diff --git a/src/Databases/MySQL/MaterializeMetadata.h b/src/Databases/MySQL/MaterializeMetadata.h index be6f83f8166..79769ef9f4a 100644 --- a/src/Databases/MySQL/MaterializeMetadata.h +++ b/src/Databases/MySQL/MaterializeMetadata.h @@ -23,11 +23,13 @@ struct MaterializeMetadata void fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & connection); - bool checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection) const; + bool checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection, const String & mysql_version) const; void transaction(const MySQLReplication::Position & position, const std::function & fun); - MaterializeMetadata(mysqlxx::PoolWithFailover::Entry & connection, const String & path, const String & database, bool & opened_transaction); + MaterializeMetadata( + mysqlxx::PoolWithFailover::Entry & connection, const String & path + , const String & database, bool & opened_transaction, const String & mysql_version); }; } diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index aa0adb3b5a0..d53ca15e4b4 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -30,6 +30,7 @@ namespace ErrorCodes { extern const int INCORRECT_QUERY; extern const int SYNTAX_ERROR; + extern const int ILLEGAL_MYSQL_VARIABLE; } static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; @@ -77,25 +78,80 @@ MaterializeMySQLSyncThread::~MaterializeMySQLSyncThread() } } +static String checkVariableAndGetVersion(const mysqlxx::Pool::Entry & connection) +{ + Block variables_header{ + {std::make_shared(), "Variable_name"}, + {std::make_shared(), "Value"} + }; + + const String & check_query = "SHOW VARIABLES WHERE " + "(variable_name = 'log_bin' AND upper(Value) = 'ON') " + "OR (variable_name = 'binlog_format' AND upper(Value) = 'ROW') " + "OR (variable_name = 'binlog_row_image' AND upper(Value) = 'FULL');"; + + MySQLBlockInputStream variables_input(connection, check_query, variables_header, DEFAULT_BLOCK_SIZE); + + Block variables_block = variables_input.read(); + if (!variables_block || variables_block.rows() != 3) + { + std::unordered_map variable_error_message{ + {"log_bin", "log_bin = 'ON'"}, + {"binlog_format", "binlog_format='ROW'"}, + {"binlog_row_image", "binlog_row_image='FULL'"} + }; + ColumnPtr variable_name_column = variables_block.getByName("Variable_name").column; + + for (size_t index = 0; index < variables_block.rows(); ++index) + { + const auto & error_message_it = variable_error_message.find(variable_name_column->getDataAt(index).toString()); + + if (error_message_it != variable_error_message.end()) + variable_error_message.erase(error_message_it); + } + + bool first = true; + std::stringstream error_message; + error_message << "Illegal MySQL variables, the MaterializeMySQL engine requires "; + for (const auto & [variable_name, variable_error_message] : variable_error_message) + { + error_message << (first ? "" : ", ") << variable_error_message; + + if (first) + first = false; + } + + throw Exception(error_message.str(), ErrorCodes::ILLEGAL_MYSQL_VARIABLE); + } + + Block version_header{{std::make_shared(), "version"}}; + MySQLBlockInputStream version_input(connection, "SELECT version() AS version;", version_header, DEFAULT_BLOCK_SIZE); + + Block version_block = version_input.read(); + if (!version_block || version_block.rows() != 1) + throw Exception("LOGICAL ERROR: cannot get mysql version.", ErrorCodes::LOGICAL_ERROR); + + return version_block.getByPosition(0).column->getDataAt(0).toString(); +} + MaterializeMySQLSyncThread::MaterializeMySQLSyncThread( const Context & context, const String & database_name_, const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, MaterializeMySQLSettings * settings_) : log(&Poco::Logger::get("MaterializeMySQLSyncThread")), global_context(context.getGlobalContext()), database_name(database_name_) , mysql_database_name(mysql_database_name_), pool(std::move(pool_)), client(std::move(client_)), settings(settings_) { - /// TODO: 做简单的check, 失败即报错 - /// binlog_format = ROW binlog_row_image = FULL + const auto & mysql_server_version = checkVariableAndGetVersion(pool.get()); query_prefix = "EXTERNAL DDL FROM MySQL(" + backQuoteIfNeed(database_name) + ", " + backQuoteIfNeed(mysql_database_name) + ") "; - startSynchronization(); + startSynchronization(mysql_server_version); } -void MaterializeMySQLSyncThread::synchronization() +void MaterializeMySQLSyncThread::synchronization(const String & mysql_version) { setThreadName(MYSQL_BACKGROUND_THREAD_NAME); try { - if (std::optional metadata = prepareSynchronized()) + if (std::optional metadata = prepareSynchronized(mysql_version)) { Stopwatch watch; Buffers buffers(database_name); @@ -150,10 +206,10 @@ void MaterializeMySQLSyncThread::stopSynchronization() } } -void MaterializeMySQLSyncThread::startSynchronization() +void MaterializeMySQLSyncThread::startSynchronization(const String & mysql_version) { /// TODO: reset exception. - background_thread_pool = std::make_unique([this]() { synchronization(); }); + background_thread_pool = std::make_unique([this, mysql_version = mysql_version]() { synchronization(mysql_version); }); } static inline void cleanOutdatedTables(const String & database_name, const Context & context) @@ -208,7 +264,7 @@ static inline UInt32 randomNumber() return dist6(rng); } -std::optional MaterializeMySQLSyncThread::prepareSynchronized() +std::optional MaterializeMySQLSyncThread::prepareSynchronized(const String & mysql_version) { std::unique_lock lock(sync_mutex); @@ -227,7 +283,9 @@ std::optional MaterializeMySQLSyncThread::prepareSynchroniz connection = pool.get(); opened_transaction = false; - MaterializeMetadata metadata(connection, getDatabase(database_name).getMetadataPath() + "/.metadata", mysql_database_name, opened_transaction); + MaterializeMetadata metadata( + connection, getDatabase(database_name).getMetadataPath() + "/.metadata", + mysql_database_name, opened_transaction, mysql_version); if (!metadata.need_dumping_tables.empty()) { diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index 76297a55f54..18a0bba7a25 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -32,7 +32,7 @@ public: void stopSynchronization(); - void startSynchronization(); + void startSynchronization(const String & mysql_version); static bool isMySQLSyncThread(); @@ -73,11 +73,11 @@ private: BufferAndSortingColumnsPtr getTableDataBuffer(const String & table, const Context & context); }; - void synchronization(); + void synchronization(const String & mysql_version); bool isCancelled() { return sync_quit.load(std::memory_order_relaxed); } - std::optional prepareSynchronized(); + std::optional prepareSynchronized(const String & mysql_version); void flushBuffersData(Buffers & buffers, MaterializeMetadata & metadata); diff --git a/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml b/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml new file mode 100644 index 00000000000..e69de29bb2d From db6220e52ce3576c4aee2f7ef4ddc224c34e449e Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 17 Jul 2020 16:24:21 +0800 Subject: [PATCH 098/374] ISSUES-4006 add mysql 5.7 ddl integration test --- .../composes/mysql_5_7_compose.yml | 10 + .../materialize_with_ddl.py | 197 +++++++++++++++++ .../test_materialize_mysql_database/test.py | 208 ++++-------------- 3 files changed, 250 insertions(+), 165 deletions(-) create mode 100644 tests/integration/test_materialize_mysql_database/materialize_with_ddl.py diff --git a/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml b/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml index e69de29bb2d..097a41b4b46 100644 --- a/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml +++ b/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml @@ -0,0 +1,10 @@ +version: '2.3' +services: + mysql1: + image: mysql:5.7 + restart: always + environment: + MYSQL_ROOT_PASSWORD: clickhouse + ports: + - 33307:3306 + command: -log-bin='mysql-bin-1.log' diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py new file mode 100644 index 00000000000..f28c7107e25 --- /dev/null +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -0,0 +1,197 @@ +import time + + +def check_query(query, result_set, retry_count=3, interval_seconds=1): + lastest_result = '' + for index in range(retry_count): + lastest_result = clickhouse_node.query(query) + + if result_set == lastest_result: + return + + time.sleep(interval_seconds) + + assert lastest_result == result_set + + +def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node): + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + + mysql_node.query("DROP TABLE test_database.test_table_1;") + + mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + + mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") + + # create mapping + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:{}', 'test_database', '{}', '{}')".format( + mysql_node.hostname, mysql_node.port, mysql_node.user, mysql_node.password)) + + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") + check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") + + mysql_node.query("INSERT INTO test_database.test_table2 VALUES(1), (2), (3), (4), (5), (6)") + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + + mysql_node.query("DROP TABLE test_database.test_table_1;") + mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") + check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") + + mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database") + + +def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node): + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + # existed before the mapping was created + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + # it already has some data + mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1)(2)(3)(5)(6)(7)") + + # create mapping + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:{}', 'test_database', '{}', '{}')".format( + mysql_node.hostname, mysql_node.port, mysql_node.user, mysql_node.password)) + + # Check for pre-existing status + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query("SELECT * FROM test_database.test_table_1 ORDER BY id FORMAT TSV", "1\n2\n3\n5\n6\n7") + + mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1)(2)(3)(4)(5)(6)") + check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6") + + mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database") + + +def rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node): + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + + mysql_node.query("RENAME TABLE test_database.test_table_1 TO test_database.test_table_2") + + # create mapping + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:{}', 'test_database', '{}', '{}')".format( + mysql_node.hostname, mysql_node.port, mysql_node.user, mysql_node.password)) + + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") + mysql_node.query("RENAME TABLE test_database.test_table_2 TO test_database.test_table_1") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + + mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database") + + +def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node): + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + + mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_1 INT NOT NULL") + mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") + mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") + mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT (id)") + + # create mapping + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:{}', 'test_database', '{}', '{}')".format( + mysql_node.hostname, mysql_node.port, mysql_node.user, mysql_node.password)) + + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query("DESC test_database.test_table_1 FORMAT TSV", + "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\nadd_column_4\tInt32\t\t\t\t") + mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t") + mysql_node.query("ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") + mysql_node.query( + "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT (id)") + check_query("DESC test_database.test_table_2 FORMAT TSV", + "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\nadd_column_4\tInt32\t\t\t\t") + + mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database") + + +def alter_drop_column_with_materialize_mysql_database(clickhouse_node, mysql_node): + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") + + mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column") + + # create mapping + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:{}', 'test_database', '{}', '{}')".format( + mysql_node.hostname, mysql_node.port, mysql_node.user, mysql_node.password)) + + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query("DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t") + mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\ndrop_column\tInt32\t\t\t\t") + mysql_node.query("ALTER TABLE test_database.test_table_2 DROP COLUMN drop_column") + check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t") + + mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database") + + +def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_node): + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + + # maybe should test rename primary key? + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") + + mysql_node.query("ALTER TABLE test_database.test_table_1 RENAME COLUMN rename_column TO new_column_name") + + # create mapping + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:{}', 'test_database', '{}', '{}')".format( + mysql_node.hostname, mysql_node.port, mysql_node.user, mysql_node.password)) + + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query("DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t") + mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nrename_column\tInt32\t\t\t\t") + mysql_node.query("ALTER TABLE test_database.test_table_2 RENAME COLUMN rename_column TO new_column_name") + check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t") + + mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database") + + +def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_node): + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + + # maybe should test rename primary key? + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") + + mysql_node.query("ALTER TABLE test_database.test_table_1 MODIFY COLUMN modify_column INT") + + # create mapping + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:{}', 'test_database', '{}', '{}')".format( + mysql_node.hostname, mysql_node.port, mysql_node.user, mysql_node.password)) + + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query("DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t") + mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") + check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tInt32\t\t\t\t") + mysql_node.query("ALTER TABLE test_database.test_table_1 MODIFY COLUMN modify_column INT") + check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t") + + mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database") + +# TODO: need support ALTER TABLE table_name ADD COLUMN column_name, RENAME COLUMN column_name TO new_column_name; +# def test_mysql_alter_change_column_for_materialize_mysql_database(started_cluster): +# pass diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index c5ad719bcd1..81548e2b8f0 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -1,13 +1,18 @@ +import os +import subprocess import time -import contextlib import pymysql.cursors import pytest +import materialize_with_ddl from helpers.cluster import ClickHouseCluster +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + cluster = ClickHouseCluster(__file__) -clickhouse_node = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], with_mysql=True) +clickhouse_node = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], with_mysql=False) + @pytest.fixture(scope="module") def started_cluster(): @@ -17,6 +22,7 @@ def started_cluster(): finally: cluster.shutdown() + class MySQLNodeInstance: def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=3308): self.user = user @@ -25,180 +31,52 @@ class MySQLNodeInstance: self.password = password self.mysql_connection = None # lazy init - def query(self, execution_query): + def alloc_connection(self): if self.mysql_connection is None: self.mysql_connection = pymysql.connect(user=self.user, password=self.password, host=self.hostname, port=self.port) - with self.mysql_connection.cursor() as cursor: + return self.mysql_connection + + def query(self, execution_query): + with self.alloc_connection().cursor() as cursor: cursor.execute(execution_query) def close(self): if self.mysql_connection is not None: self.mysql_connection.close() + def wait_mysql_to_start(self, timeout=60): + start = time.time() + while time.time() - start < timeout: + try: + self.alloc_connection() + print "Mysql Started" + return + except Exception as ex: + print "Can't connect to MySQL " + str(ex) + time.sleep(0.5) -def check_query(query, result_set, retry_count=3, interval_seconds=1): - for index in range(retry_count): - if result_set == clickhouse_node.query(query): - return - time.sleep(interval_seconds) - raise Exception("") + subprocess.check_call(['docker-compose', 'ps', '--services', '--all']) + raise Exception("Cannot wait MySQL container") -def test_mysql_drop_table_for_materialize_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") +@pytest.fixture(scope="module") +def started_mysql_5_7(): + mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33307) - mysql_node.query("DROP TABLE test_database.test_table_1;") - - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - - mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") - - # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") - - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") - check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") - - mysql_node.query("INSERT INTO test_database.test_table2 VALUES(1)(2)(3)(4)(5)(6)") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") - - mysql_node.query("DROP TABLE test_database.test_table_1;") - mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") - check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") + try: + docker_compose = os.path.join(SCRIPT_DIR, 'composes', 'mysql_5_7_compose.yml') + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d']) + mysql_node.wait_mysql_to_start(120) + yield mysql_node + finally: + mysql_node.close() -def test_mysql_create_table_for_materialize_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - # existed before the mapping was created - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - # it already has some data - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1)(2)(3)(5)(6)(7)") - - # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") - - # Check for pre-existing status - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query("SELECT * FROM test_database.test_table_1 ORDER BY id FORMAT TSV", "1\n2\n3\n5\n6\n7") - - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1)(2)(3)(4)(5)(6)") - check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6") - - -def test_mysql_rename_table_for_materialize_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - - mysql_node.query("RENAME TABLE test_database.test_table_1 TO test_database.test_table_2") - - # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") - - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") - mysql_node.query("RENAME TABLE test_database.test_table_2 TO test_database.test_table_1") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - - -def test_mysql_alter_add_column_for_materialize_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_1 INT NOT NULL") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT (id)") - - # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") - - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query("DESC test_database.test_table_1 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\nadd_column_4\tInt32\t\t\t\t") - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") - check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t") - mysql_node.query("ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") - mysql_node.query("ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT (id)") - check_query("DESC test_database.test_table_2 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\nadd_column_4\tInt32\t\t\t\t") - - -def test_mysql_alter_drop_column_for_materialize_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") - - mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column") - - # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") - - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query("DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t") - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") - check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\ndrop_column\tInt32\t\t\t\t") - mysql_node.query("ALTER TABLE test_database.test_table_2 DROP COLUMN drop_column") - check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t") - - -def test_mysql_alter_rename_column_for_materialize_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - - # maybe should test rename primary key? - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") - - mysql_node.query("ALTER TABLE test_database.test_table_1 RENAME COLUMN rename_column TO new_column_name") - - # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") - - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query("DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t") - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") - check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nrename_column\tInt32\t\t\t\t") - mysql_node.query("ALTER TABLE test_database.test_table_2 RENAME COLUMN rename_column TO new_column_name") - check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t") - - -def test_mysql_alter_modify_column_for_materialize_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - - # maybe should test rename primary key? - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") - - mysql_node.query("ALTER TABLE test_database.test_table_1 MODIFY COLUMN modify_column INT") - - # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") - - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query("DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t") - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") - check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tInt32\t\t\t\t") - mysql_node.query("ALTER TABLE test_database.test_table_1 MODIFY COLUMN modify_column INT") - check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t") - - -# TODO: need support ALTER TABLE table_name ADD COLUMN column_name, RENAME COLUMN column_name TO new_column_name; -# def test_mysql_alter_change_column_for_materialize_mysql_database(started_cluster): -# pass +def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7): + materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7) + materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7) + materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7) + materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7) + materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7) + materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7) + materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7) From ad187384f0a73371893abc58aff03680a69174f9 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 17 Jul 2020 16:37:27 +0800 Subject: [PATCH 099/374] ISSUES-4006 try fix build failure --- src/Core/MySQLReplication.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 6dd507ec409..203d938952b 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include namespace DB From 2fd0c95d790aa25f331ad8356c291be0144d9b5f Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 17 Jul 2020 18:05:46 +0800 Subject: [PATCH 100/374] ISSUES-4006 try fix integration test --- .../composes/mysql_5_7_compose.yml | 2 +- .../materialize_with_ddl.py | 87 +++++++++---------- .../test_materialize_mysql_database/test.py | 5 +- 3 files changed, 44 insertions(+), 50 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml b/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml index 097a41b4b46..91ee5ce60c4 100644 --- a/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml +++ b/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml @@ -1,6 +1,6 @@ version: '2.3' services: - mysql1: + mysql5_7: image: mysql:5.7 restart: always environment: diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index f28c7107e25..8c7b74b2221 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -1,7 +1,7 @@ import time -def check_query(query, result_set, retry_count=3, interval_seconds=1): +def check_query(clickhouse_node, query, result_set, retry_count=3, interval_seconds=1): lastest_result = '' for index in range(retry_count): lastest_result = clickhouse_node.query(query) @@ -25,22 +25,21 @@ def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node): mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:{}', 'test_database', '{}', '{}')".format( - mysql_node.hostname, mysql_node.port, mysql_node.user, mysql_node.password)) + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") - check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") mysql_node.query("INSERT INTO test_database.test_table2 VALUES(1), (2), (3), (4), (5), (6)") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") mysql_node.query("DROP TABLE test_database.test_table_1;") mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") - check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") @@ -54,18 +53,17 @@ def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node): mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1)(2)(3)(5)(6)(7)") # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:{}', 'test_database', '{}', '{}')".format( - mysql_node.hostname, mysql_node.port, mysql_node.user, mysql_node.password)) + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") # Check for pre-existing status assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query("SELECT * FROM test_database.test_table_1 ORDER BY id FORMAT TSV", "1\n2\n3\n5\n6\n7") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY id FORMAT TSV", "1\n2\n3\n5\n6\n7") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1)(2)(3)(4)(5)(6)") - check_query("SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") @@ -78,13 +76,12 @@ def rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node): mysql_node.query("RENAME TABLE test_database.test_table_1 TO test_database.test_table_2") # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:{}', 'test_database', '{}', '{}')".format( - mysql_node.hostname, mysql_node.port, mysql_node.user, mysql_node.password)) + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") mysql_node.query("RENAME TABLE test_database.test_table_2 TO test_database.test_table_1") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") @@ -100,20 +97,19 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT (id)") # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:{}', 'test_database', '{}', '{}')".format( - mysql_node.hostname, mysql_node.port, mysql_node.user, mysql_node.password)) + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query("DESC test_database.test_table_1 FORMAT TSV", + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\nadd_column_4\tInt32\t\t\t\t") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") - check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t") mysql_node.query("ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query( "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT (id)") - check_query("DESC test_database.test_table_2 FORMAT TSV", + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\nadd_column_4\tInt32\t\t\t\t") mysql_node.query("DROP DATABASE test_database") @@ -127,17 +123,16 @@ def alter_drop_column_with_materialize_mysql_database(clickhouse_node, mysql_nod mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column") # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:{}', 'test_database', '{}', '{}')".format( - mysql_node.hostname, mysql_node.port, mysql_node.user, mysql_node.password)) + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query("DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") - check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\ndrop_column\tInt32\t\t\t\t") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\ndrop_column\tInt32\t\t\t\t") mysql_node.query("ALTER TABLE test_database.test_table_2 DROP COLUMN drop_column") - check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") @@ -152,17 +147,16 @@ def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_n mysql_node.query("ALTER TABLE test_database.test_table_1 RENAME COLUMN rename_column TO new_column_name") # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:{}', 'test_database', '{}', '{}')".format( - mysql_node.hostname, mysql_node.port, mysql_node.user, mysql_node.password)) + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query("DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") - check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nrename_column\tInt32\t\t\t\t") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nrename_column\tInt32\t\t\t\t") mysql_node.query("ALTER TABLE test_database.test_table_2 RENAME COLUMN rename_column TO new_column_name") - check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") @@ -177,17 +171,16 @@ def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_n mysql_node.query("ALTER TABLE test_database.test_table_1 MODIFY COLUMN modify_column INT") # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:{}', 'test_database', '{}', '{}')".format( - mysql_node.hostname, mysql_node.port, mysql_node.user, mysql_node.password)) + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query("DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") - check_query("SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") - check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tInt32\t\t\t\t") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tInt32\t\t\t\t") mysql_node.query("ALTER TABLE test_database.test_table_1 MODIFY COLUMN modify_column INT") - check_query("DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 81548e2b8f0..df42decb146 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -55,21 +55,22 @@ class MySQLNodeInstance: print "Can't connect to MySQL " + str(ex) time.sleep(0.5) - subprocess.check_call(['docker-compose', 'ps', '--services', '--all']) + subprocess.check_call(['docker-compose', 'ps', '--services', 'all']) raise Exception("Cannot wait MySQL container") @pytest.fixture(scope="module") def started_mysql_5_7(): mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33307) + docker_compose = os.path.join(SCRIPT_DIR, 'composes', 'mysql_5_7_compose.yml') try: - docker_compose = os.path.join(SCRIPT_DIR, 'composes', 'mysql_5_7_compose.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d']) mysql_node.wait_mysql_to_start(120) yield mysql_node finally: mysql_node.close() + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes', '--remove-orphans']) def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7): From 0974f6c6926489f2d70265aef765f4464b8676ae Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 17 Jul 2020 18:27:34 +0800 Subject: [PATCH 101/374] ISSUES-4006 try fix integration test --- .../composes/mysql_5_7_compose.yml | 2 +- .../materialize_with_ddl.py | 63 ++++++++++--------- 2 files changed, 33 insertions(+), 32 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml b/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml index 91ee5ce60c4..1acf6c64876 100644 --- a/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml +++ b/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml @@ -7,4 +7,4 @@ services: MYSQL_ROOT_PASSWORD: clickhouse ports: - 33307:3306 - command: -log-bin='mysql-bin-1.log' + command: --server_id=100 --log-bin='mysql-bin-1.log' diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 8c7b74b2221..68acc7d7298 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -9,6 +9,7 @@ def check_query(clickhouse_node, query, result_set, retry_count=3, interval_seco if result_set == lastest_result: return + print lastest_result time.sleep(interval_seconds) assert lastest_result == result_set @@ -28,17 +29,17 @@ def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node): clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2\n") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") - mysql_node.query("INSERT INTO test_database.test_table2 VALUES(1), (2), (3), (4), (5), (6)") + mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1), (2), (3), (4), (5), (6)") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6\n") mysql_node.query("DROP TABLE test_database.test_table_1;") mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2\n") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") mysql_node.query("DROP DATABASE test_database") @@ -57,13 +58,13 @@ def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node): # Check for pre-existing status assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY id FORMAT TSV", "1\n2\n3\n5\n6\n7") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY id FORMAT TSV", "1\n2\n3\n5\n6\n7\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1)(2)(3)(4)(5)(6)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6\n") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") @@ -79,9 +80,9 @@ def rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node): clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2\n") mysql_node.query("RENAME TABLE test_database.test_table_2 TO test_database.test_table_1") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") @@ -100,17 +101,17 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", - "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\nadd_column_4\tInt32\t\t\t\t") + "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\nadd_column_4\tInt32\t\t\t\t\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query( "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT (id)") check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", - "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\nadd_column_4\tInt32\t\t\t\t") + "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\nadd_column_4\tInt32\t\t\t\t\n") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") @@ -126,13 +127,13 @@ def alter_drop_column_with_materialize_mysql_database(clickhouse_node, mysql_nod clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\ndrop_column\tInt32\t\t\t\t") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\ndrop_column\tInt32\t\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_2 DROP COLUMN drop_column") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\n") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") @@ -150,13 +151,13 @@ def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_n clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nrename_column\tInt32\t\t\t\t") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nrename_column\tInt32\t\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_2 RENAME COLUMN rename_column TO new_column_name") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\n") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") @@ -174,13 +175,13 @@ def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_n clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tInt32\t\t\t\t") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tInt32\t\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_1 MODIFY COLUMN modify_column INT") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\n") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") From 9ba28d41ca1433bc711201e958eff1de4a7137aa Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 17 Jul 2020 19:00:34 +0800 Subject: [PATCH 102/374] ISSUES-4006 skip drop database query for materialize mysql engine --- src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 527738d143f..b0b01a24e55 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -298,7 +298,8 @@ ASTPtr InterpreterDropImpl::getRewrittenQuery( { const auto & database_name = context.resolveDatabase(drop_query.database); - if (database_name != filter_mysql_db) + /// Skip drop databse|view|dictionary + if (database_name != filter_mysql_db || drop_query.table.empty() || drop_query.is_view || drop_query.is_dictionary) return {}; ASTPtr rewritten_query = drop_query.clone(); From c5946f7697406d11f8933eef9cc881908aecb7cb Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 18 Jul 2020 12:13:41 +0800 Subject: [PATCH 103/374] ISSUES-4006 add mysql 8.0 integration test --- .../AddingVersionsBlockOutputStream.cpp | 3 + .../AddingVersionsBlockOutputStream.h | 6 ++ .../MySQL/DatabaseMaterializeMySQL.cpp | 8 ++- .../MySQL/MaterializeMySQLSyncThread.cpp | 9 ++- .../MySQL/InterpretersMySQLDDLQuery.cpp | 7 +- .../composes/mysql_8_0_compose.yml | 10 +++ .../materialize_with_ddl.py | 64 +++++++++++-------- .../test_materialize_mysql_database/test.py | 39 +++++++++-- 8 files changed, 107 insertions(+), 39 deletions(-) create mode 100644 tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml diff --git a/src/DataStreams/AddingVersionsBlockOutputStream.cpp b/src/DataStreams/AddingVersionsBlockOutputStream.cpp index c2c83e6adc9..b94c858b38e 100644 --- a/src/DataStreams/AddingVersionsBlockOutputStream.cpp +++ b/src/DataStreams/AddingVersionsBlockOutputStream.cpp @@ -39,6 +39,9 @@ void AddingVersionsBlockOutputStream::write(const Block & block) res.insert({sign_column, sign_type, header.getByPosition(header.columns() - 2).name}); res.insert({version_column, version_type, header.getByPosition(header.columns() - 1).name}); output->write(res); + + written_rows += block.rows(); + written_bytes += block.bytes(); } Block AddingVersionsBlockOutputStream::getHeader() const { diff --git a/src/DataStreams/AddingVersionsBlockOutputStream.h b/src/DataStreams/AddingVersionsBlockOutputStream.h index cd6eb84e9bc..4edc0ac13c7 100644 --- a/src/DataStreams/AddingVersionsBlockOutputStream.h +++ b/src/DataStreams/AddingVersionsBlockOutputStream.h @@ -25,6 +25,12 @@ public: private: size_t & version; BlockOutputStreamPtr output; + + std::atomic written_rows{0}, written_bytes{0}; + +public: + size_t getWrittenRows() { return written_rows; } + size_t getWrittenBytes() { return written_bytes; } }; } diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 62b6c41dbb8..db657edefed 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -168,7 +168,13 @@ void DatabaseMaterializeMySQL::renameTable(const Context & context, const String if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) throw Exception("MaterializeMySQL database not support rename table.", ErrorCodes::NOT_IMPLEMENTED); - getNestedDatabase()->renameTable(context, name, to_database, to_name, exchange); + if (exchange) + throw Exception("MaterializeMySQL database not support exchange table.", ErrorCodes::NOT_IMPLEMENTED); + + if (to_database.getDatabaseName() != getDatabaseName()) + throw Exception("Cannot rename with other database for MaterializeMySQL database.", ErrorCodes::NOT_IMPLEMENTED); + + getNestedDatabase()->renameTable(context, name, *getNestedDatabase(), to_name, exchange); } void DatabaseMaterializeMySQL::alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index d53ca15e4b4..e4bcdddb791 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -248,11 +248,18 @@ static inline void dumpDataForTables( String comment = "Materialize MySQL step 1: execute MySQL DDL for dump data"; tryToExecuteQuery(query_prefix + " " + iterator->second, context, mysql_database_name, comment); /// create table. - BlockOutputStreamPtr out = std::make_shared(master_info.version, getTableOutput(database_name, table_name, context)); + auto out = std::make_shared(master_info.version, getTableOutput(database_name, table_name, context)); MySQLBlockInputStream input( connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name), out->getHeader(), DEFAULT_BLOCK_SIZE); + + Stopwatch watch; copyData(input, *out, is_cancelled); + LOG_INFO(&Poco::Logger::get("MaterializeMySQLSyncThread(" + database_name + ")"), + "Materialize MySQL step 1: dump {}, {} rows, {} in {} sec., {} rows/sec., {}/sec.", + table_name, out->getWrittenRows(), ReadableSize(out->getWrittenBytes()), watch.elapsedSeconds(), + static_cast(out->getWrittenRows() / watch.elapsedSeconds()), + ReadableSize(out->getWrittenRows() / watch.elapsedSeconds())); } } diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index b0b01a24e55..4afa651a828 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -298,7 +298,7 @@ ASTPtr InterpreterDropImpl::getRewrittenQuery( { const auto & database_name = context.resolveDatabase(drop_query.database); - /// Skip drop databse|view|dictionary + /// Skip drop database|view|dictionary if (database_name != filter_mysql_db || drop_query.table.empty() || drop_query.is_view || drop_query.is_dictionary) return {}; @@ -458,7 +458,10 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( throw Exception("It is a bug", ErrorCodes::LOGICAL_ERROR); new_column_name = modify_columns.front().name; - modify_columns.front().name = alter_command->old_name; + + if (!alter_command->old_name.empty()) + modify_columns.front().name = alter_command->old_name; + rewritten_command->col_decl = InterpreterCreateQuery::formatColumns(modify_columns)->children[0]; if (!alter_command->column_name.empty()) diff --git a/tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml b/tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml new file mode 100644 index 00000000000..b5583bb5173 --- /dev/null +++ b/tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml @@ -0,0 +1,10 @@ +version: '2.3' +services: + mysql8_0: + image: mysql:8.0 + restart: always + environment: + MYSQL_ROOT_PASSWORD: clickhouse + ports: + - 33308:3306 + command: --server_id=100 --log-bin='mysql-bin-1.log' diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 68acc7d7298..b6651523481 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -1,7 +1,7 @@ import time -def check_query(clickhouse_node, query, result_set, retry_count=3, interval_seconds=1): +def check_query(clickhouse_node, query, result_set, retry_count=3, interval_seconds=3): lastest_result = '' for index in range(retry_count): lastest_result = clickhouse_node.query(query) @@ -15,7 +15,7 @@ def check_query(clickhouse_node, query, result_set, retry_count=3, interval_seco assert lastest_result == result_set -def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node): +def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") @@ -26,7 +26,8 @@ def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node): mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2\n") @@ -46,15 +47,15 @@ def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node): clickhouse_node.query("DROP DATABASE test_database") -def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node): +def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") # existed before the mapping was created mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") # it already has some data - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1)(2)(3)(5)(6)(7)") + mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1), (2), (3), (5), (6), (7);") # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) # Check for pre-existing status assert "test_database" in clickhouse_node.query("SHOW DATABASES") @@ -70,14 +71,15 @@ def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node): clickhouse_node.query("DROP DATABASE test_database") -def rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node): +def rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") mysql_node.query("RENAME TABLE test_database.test_table_1 TO test_database.test_table_2") # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2\n") @@ -88,58 +90,62 @@ def rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node): clickhouse_node.query("DROP DATABASE test_database") -def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node): +def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_1 INT NOT NULL") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT (id)") + mysql_node.query( + "ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ("0" if service_name == "mysql5_7" else "(id)")) # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", - "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\nadd_column_4\tInt32\t\t\t\t\n") + "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t\t\t\t\t\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query( - "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT (id)") + "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( + "0" if service_name == "mysql5_7" else "(id)")) check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", - "add_column_2\tInt32\t\t\t\t\nid\tInt32\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\nadd_column_4\tInt32\t\t\t\t\n") + "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t\t\t\t\t\n") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") -def alter_drop_column_with_materialize_mysql_database(clickhouse_node, mysql_node): +def alter_drop_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column") # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\ndrop_column\tInt32\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\ndrop_column\tInt32\t\t\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_2 DROP COLUMN drop_column") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\n") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") -def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_node): +def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") # maybe should test rename primary key? @@ -148,7 +154,8 @@ def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_n mysql_node.query("ALTER TABLE test_database.test_table_1 RENAME COLUMN rename_column TO new_column_name") # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") @@ -163,7 +170,7 @@ def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_n clickhouse_node.query("DROP DATABASE test_database") -def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_node): +def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") # maybe should test rename primary key? @@ -172,16 +179,17 @@ def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_n mysql_node.query("ALTER TABLE test_database.test_table_1 MODIFY COLUMN modify_column INT") # create mapping - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('mysql5_7:3306', 'test_database', 'root', 'clickhouse')") + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tInt32\t\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_1 MODIFY COLUMN modify_column INT") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tInt32\t\t\t\t\t\n") + mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index df42decb146..62908f07b7a 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -73,11 +73,36 @@ def started_mysql_5_7(): subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes', '--remove-orphans']) +@pytest.fixture(scope="module") +def started_mysql_8_0(): + mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33308) + docker_compose = os.path.join(SCRIPT_DIR, 'composes', 'mysql_8_0_compose.yml') + + try: + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d']) + mysql_node.wait_mysql_to_start(120) + yield mysql_node + finally: + mysql_node.close() + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes', '--remove-orphans']) + + def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7): - materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7) - materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7) - materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7) - materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7) - materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7) - materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7) - materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7) + materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + # mysql 5.7 cannot support alter rename column + # materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + + +def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_8_0): + materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") From 9d8234afd7d496b2aba5a10119a234c5c0d1a3ce Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 18 Jul 2020 20:06:12 +0800 Subject: [PATCH 104/374] ISSUES-4006 try fix integration test --- src/Databases/MySQL/MaterializeMySQLSyncThread.cpp | 12 +++++++----- src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 2 +- .../composes/mysql_8_0_compose.yml | 2 +- .../materialize_with_ddl.py | 6 ++++-- .../test_materialize_mysql_database/test.py | 2 +- 5 files changed, 14 insertions(+), 10 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index e4bcdddb791..0b8e39054bc 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -86,19 +86,21 @@ static String checkVariableAndGetVersion(const mysqlxx::Pool::Entry & connection }; const String & check_query = "SHOW VARIABLES WHERE " - "(variable_name = 'log_bin' AND upper(Value) = 'ON') " - "OR (variable_name = 'binlog_format' AND upper(Value) = 'ROW') " - "OR (variable_name = 'binlog_row_image' AND upper(Value) = 'FULL');"; + "(Variable_name = 'log_bin' AND upper(Value) = 'ON') " + "OR (Variable_name = 'binlog_format' AND upper(Value) = 'ROW') " + "OR (Variable_name = 'binlog_row_image' AND upper(Value) = 'FULL') " + "OR (Variable_name = 'default_authentication_plugin' AND upper(Value) = 'MYSQL_NATIVE_PASSWORD');"; MySQLBlockInputStream variables_input(connection, check_query, variables_header, DEFAULT_BLOCK_SIZE); Block variables_block = variables_input.read(); - if (!variables_block || variables_block.rows() != 3) + if (!variables_block || variables_block.rows() != 4) { std::unordered_map variable_error_message{ {"log_bin", "log_bin = 'ON'"}, {"binlog_format", "binlog_format='ROW'"}, - {"binlog_row_image", "binlog_row_image='FULL'"} + {"binlog_row_image", "binlog_row_image='FULL'"}, + {"default_authentication_plugin", "default_authentication_plugin='mysql_native_password'"} }; ColumnPtr variable_name_column = variables_block.getByName("Variable_name").column; diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 4afa651a828..79d0ea16d20 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -360,6 +360,7 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( rewritten_query->table = alter_query.table; rewritten_query->set(rewritten_query->command_list, std::make_shared()); + String default_after_column; for (const auto & command_query : alter_query.command_list->children) { const auto & alter_command = command_query->as(); @@ -369,7 +370,6 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( const auto & additional_columns_name_and_type = getColumnsList(alter_command->additional_columns); const auto & additional_columns = InterpreterCreateQuery::formatColumns(additional_columns_name_and_type); - String default_after_column; for (size_t index = 0; index < additional_columns_name_and_type.size(); ++index) { auto rewritten_command = std::make_shared(); diff --git a/tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml b/tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml index b5583bb5173..5efda3252c6 100644 --- a/tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml +++ b/tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml @@ -7,4 +7,4 @@ services: MYSQL_ROOT_PASSWORD: clickhouse ports: - 33308:3306 - command: --server_id=100 --log-bin='mysql-bin-1.log' + command: --server_id=100 --log-bin='mysql-bin-1.log' --default_authentication_plugin='mysql_native_password' diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index b6651523481..dcbdb8ed9ef 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -63,7 +63,7 @@ def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node, se check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY id FORMAT TSV", "1\n2\n3\n5\n6\n7\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1)(2)(3)(4)(5)(6)") + mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1), (2), (3), (4), (5), (6);") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6\n") @@ -115,8 +115,10 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node mysql_node.query( "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( "0" if service_name == "mysql5_7" else "(id)")) + + default_expression = "DEFAULT\t0" if service_name == "mysql5_7" else "DEFAULT\t(id)" check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", - "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t\t\t\t\t\n") + "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t" + default_expression + "\t\t\t\n") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 62908f07b7a..03b2d15d404 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -33,7 +33,7 @@ class MySQLNodeInstance: def alloc_connection(self): if self.mysql_connection is None: - self.mysql_connection = pymysql.connect(user=self.user, password=self.password, host=self.hostname, port=self.port) + self.mysql_connection = pymysql.connect(user=self.user, password=self.password, host=self.hostname, port=self.port, autocommit=True) return self.mysql_connection def query(self, execution_query): From cfe2c42199b3037f17bfaa359fd2129543caa515 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 18 Jul 2020 20:25:45 +0800 Subject: [PATCH 105/374] ISSUES-4006 try fix integration test --- .../test_materialize_mysql_database/materialize_with_ddl.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index dcbdb8ed9ef..866c42722be 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -116,7 +116,7 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( "0" if service_name == "mysql5_7" else "(id)")) - default_expression = "DEFAULT\t0" if service_name == "mysql5_7" else "DEFAULT\t(id)" + default_expression = "DEFAULT\t0" if service_name == "mysql5_7" else "DEFAULT\tid" check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t" + default_expression + "\t\t\t\n") @@ -192,6 +192,10 @@ def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_n check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tInt32\t\t\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT") check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n") + mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT FIRST") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "modify_column\tNullable(Int32)\t\t\t\t\t\nid\tInt32\t\t\t\t\t\n") + mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT AFTER id") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") From d57631210e547da10fc3f4f69b6932d2e6b43990 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 18 Jul 2020 20:32:30 +0800 Subject: [PATCH 106/374] ISSUES-4006 try fix integration test --- .../test_materialize_mysql_database/materialize_with_ddl.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 866c42722be..e9309b4075b 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -161,12 +161,12 @@ def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_n assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\t\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nrename_column\tInt32\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nrename_column\tInt32\t\t\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_2 RENAME COLUMN rename_column TO new_column_name") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\t\n") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") From bd65a8f2135a653c6c1093532e18d9f20724f88b Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 19 Jul 2020 20:53:09 +0800 Subject: [PATCH 107/374] ISSUES-4006 try fix build failure --- src/Databases/MySQL/MaterializeMySQLSyncThread.cpp | 10 +++++----- .../materialize_with_ddl.py | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 0b8e39054bc..ba80c2c38f0 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -96,7 +96,7 @@ static String checkVariableAndGetVersion(const mysqlxx::Pool::Entry & connection Block variables_block = variables_input.read(); if (!variables_block || variables_block.rows() != 4) { - std::unordered_map variable_error_message{ + std::unordered_map variables_error_message{ {"log_bin", "log_bin = 'ON'"}, {"binlog_format", "binlog_format='ROW'"}, {"binlog_row_image", "binlog_row_image='FULL'"}, @@ -106,16 +106,16 @@ static String checkVariableAndGetVersion(const mysqlxx::Pool::Entry & connection for (size_t index = 0; index < variables_block.rows(); ++index) { - const auto & error_message_it = variable_error_message.find(variable_name_column->getDataAt(index).toString()); + const auto & error_message_it = variables_error_message.find(variable_name_column->getDataAt(index).toString()); - if (error_message_it != variable_error_message.end()) - variable_error_message.erase(error_message_it); + if (error_message_it != variables_error_message.end()) + variables_error_message.erase(error_message_it); } bool first = true; std::stringstream error_message; error_message << "Illegal MySQL variables, the MaterializeMySQL engine requires "; - for (const auto & [variable_name, variable_error_message] : variable_error_message) + for (const auto & [variable_name, variable_error_message] : variables_error_message) { error_message << (first ? "" : ", ") << variable_error_message; diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index e9309b4075b..2e12c591612 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -200,6 +200,6 @@ def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_n mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") -# TODO: need support ALTER TABLE table_name ADD COLUMN column_name, RENAME COLUMN column_name TO new_column_name; +# TODO: need ClickHouse support ALTER TABLE table_name ADD COLUMN column_name, RENAME COLUMN column_name TO new_column_name; # def test_mysql_alter_change_column_for_materialize_mysql_database(started_cluster): # pass From 7eb232e02d29cb1f66f0c78824a418da82aafdd0 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 19 Jul 2020 20:59:31 +0800 Subject: [PATCH 108/374] ISSUES-4006 add dml test after ddl --- .../materialize_with_ddl.py | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 2e12c591612..16ee46b5f23 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -107,7 +107,7 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", - "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t\t\t\t\t\n") + "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t\t\t\t\t\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\n") @@ -120,6 +120,9 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t" + default_expression + "\t\t\t\n") + mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2, 3, 4, 5), (6, 7, 8, 9, 10)") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\t2\t3\t4\t5\n6\t7\t8\t9\t10\n") + mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") @@ -143,6 +146,9 @@ def alter_drop_column_with_materialize_mysql_database(clickhouse_node, mysql_nod mysql_node.query("ALTER TABLE test_database.test_table_2 DROP COLUMN drop_column") check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\n") + mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1), (2), (3), (4), (5)") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") + mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") @@ -168,6 +174,9 @@ def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_n mysql_node.query("ALTER TABLE test_database.test_table_2 RENAME COLUMN rename_column TO new_column_name") check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\t\n") + mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2), (3, 4), (5, 6), (7, 8), (9, 10)") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\t4\n5\t6\n7\t8\n9\t10\n") + mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") @@ -197,6 +206,9 @@ def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_n mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT AFTER id") check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n") + mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2), (3, NULL)") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\tNULL\n") + mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") From 9996166860b1117735d31abbbe0a8154dbbd6a7d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 19 Jul 2020 21:20:16 +0800 Subject: [PATCH 109/374] ISSUES-4006 try fix check style --- src/Core/MySQLProtocol.h | 1 + src/Databases/MySQL/MaterializeMetadata.cpp | 5 +++++ src/Databases/MySQL/MaterializeMySQLSyncThread.cpp | 6 +++--- src/Interpreters/InterpreterFactory.cpp | 1 - src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 1 + src/Parsers/MySQL/ASTDeclareTableOptions.cpp | 2 +- src/Parsers/MySQL/tests/gtest_create_parser.cpp | 2 +- .../test_materialize_mysql_database/materialize_with_ddl.py | 2 +- 8 files changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 694e6d24baf..fb17b48fb0d 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -40,6 +40,7 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int CANNOT_WRITE_AFTER_END_OF_BUFFER; extern const int UNKNOWN_PACKET_FROM_CLIENT; extern const int MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES; diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 1879a1fb79d..87540648876 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -13,6 +13,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static std::unordered_map fetchTablesCreateQuery( const mysqlxx::PoolWithFailover::Entry & connection, const String & database_name, const std::vector & fetch_tables) { diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index ba80c2c38f0..447fc6c2e27 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -28,8 +28,8 @@ namespace DB namespace ErrorCodes { - extern const int INCORRECT_QUERY; extern const int SYNTAX_ERROR; + extern const int LOGICAL_ERROR; extern const int ILLEGAL_MYSQL_VARIABLE; } @@ -323,7 +323,7 @@ std::optional MaterializeMySQLSyncThread::prepareSynchroniz { throw; } - catch (mysqlxx::Exception & ) + catch (mysqlxx::Exception &) { /// Avoid busy loop when MySQL is not available. sleepForMilliseconds(settings->max_wait_time_when_mysql_unavailable); @@ -522,7 +522,7 @@ void MaterializeMySQLSyncThread::Buffers::commit(const Context & context) total_blocks_rows = 0; total_blocks_bytes = 0; } - catch(...) + catch (...) { data.clear(); throw; diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 6dc1e732bcf..a18d3ab8a6f 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -75,7 +75,6 @@ #include - namespace ProfileEvents { extern const Event Query; diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 79d0ea16d20..d0cf5f2bedb 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -28,6 +28,7 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_TYPE; + extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; extern const int EMPTY_LIST_OF_COLUMNS_PASSED; } diff --git a/src/Parsers/MySQL/ASTDeclareTableOptions.cpp b/src/Parsers/MySQL/ASTDeclareTableOptions.cpp index 5ed3dad1ae9..87b99cdf1ac 100644 --- a/src/Parsers/MySQL/ASTDeclareTableOptions.cpp +++ b/src/Parsers/MySQL/ASTDeclareTableOptions.cpp @@ -21,7 +21,7 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override { - if constexpr(allow_default) + if constexpr (allow_default) { if (ParserKeyword("DEFAULT").ignore(pos, expected)) { diff --git a/src/Parsers/MySQL/tests/gtest_create_parser.cpp b/src/Parsers/MySQL/tests/gtest_create_parser.cpp index 1b30a36d84a..92c0070aa88 100644 --- a/src/Parsers/MySQL/tests/gtest_create_parser.cpp +++ b/src/Parsers/MySQL/tests/gtest_create_parser.cpp @@ -35,7 +35,7 @@ TEST(CreateTableParser, SimpleCreate) TEST(CreateTableParser, SS) { ParserCreateQuery p_create_query; - String input = "CREATE TABLE `test_table_1` (`a` int DEFAULT NULL, `b` int DEFAULT NULL) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci"; + String input = "CREATE TABLE `test_table_1` (`a` int DEFAULT NULL, `b` int DEFAULT NULL) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci"; ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); ast->dumpTree(std::cerr); diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 16ee46b5f23..1dc5827e2b1 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -207,7 +207,7 @@ def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_n check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n") mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2), (3, NULL)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\tNULL\n") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\t\\N\n") mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") From b76f218d904db35f8c3f73d4e0ee467f6b86d2d4 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 21 Jul 2020 01:44:38 +0800 Subject: [PATCH 110/374] ISSUES-4006 add mysql dml test --- src/Core/MySQLReplication.cpp | 4 +- src/DataTypes/DataTypesNumber.cpp | 4 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 137 +++++++++++++++--- .../MySQL/InterpretersMySQLDDLQuery.cpp | 96 +++++++++--- .../MySQL/tests/gtest_create_rewritten.cpp | 59 ++++++++ src/Parsers/MySQL/ASTDeclareColumn.cpp | 2 + .../materialize_with_ddl.py | 68 +++++++++ .../test_materialize_mysql_database/test.py | 9 ++ 8 files changed, 335 insertions(+), 44 deletions(-) create mode 100644 src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 203d938952b..f7a817f8124 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -385,9 +385,9 @@ namespace MySQLReplication break; } case MYSQL_TYPE_FLOAT: { - Float64 val = 0; + Float32 val = 0; payload.readStrict(reinterpret_cast(&val), 4); - row.push_back(Field{Float64{val}}); + row.push_back(Field{Float32{val}}); break; } case MYSQL_TYPE_DOUBLE: { diff --git a/src/DataTypes/DataTypesNumber.cpp b/src/DataTypes/DataTypesNumber.cpp index 06b747e1d08..883dafdf5b0 100644 --- a/src/DataTypes/DataTypesNumber.cpp +++ b/src/DataTypes/DataTypesNumber.cpp @@ -62,6 +62,7 @@ void registerDataTypeNumbers(DataTypeFactory & factory) factory.registerAlias("REAL", "Float32", DataTypeFactory::CaseInsensitive); factory.registerAlias("SINGLE", "Float32", DataTypeFactory::CaseInsensitive); /// MS Access factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::CaseInsensitive); + factory.registerAlias("MEDIUMINT", "Int32", DataTypeFactory::CaseInsensitive); /// MySQL factory.registerAlias("DOUBLE PRECISION", "Float64", DataTypeFactory::CaseInsensitive); @@ -69,16 +70,17 @@ void registerDataTypeNumbers(DataTypeFactory & factory) factory.registerAlias("TINYINT SIGNED", "Int8", DataTypeFactory::CaseInsensitive); factory.registerAlias("INT1 SIGNED", "Int8", DataTypeFactory::CaseInsensitive); factory.registerAlias("SMALLINT SIGNED", "Int16", DataTypeFactory::CaseInsensitive); + factory.registerAlias("MEDIUMINT SIGNED", "Int32", DataTypeFactory::CaseInsensitive); factory.registerAlias("INT SIGNED", "Int32", DataTypeFactory::CaseInsensitive); factory.registerAlias("INTEGER SIGNED", "Int32", DataTypeFactory::CaseInsensitive); factory.registerAlias("BIGINT SIGNED", "Int64", DataTypeFactory::CaseInsensitive); factory.registerAlias("TINYINT UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive); factory.registerAlias("INT1 UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive); factory.registerAlias("SMALLINT UNSIGNED", "UInt16", DataTypeFactory::CaseInsensitive); + factory.registerAlias("MEDIUMINT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive); factory.registerAlias("INT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive); factory.registerAlias("INTEGER UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive); factory.registerAlias("BIGINT UNSIGNED", "UInt64", DataTypeFactory::CaseInsensitive); - } } diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 447fc6c2e27..e1f07fe7373 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -30,6 +30,7 @@ namespace ErrorCodes { extern const int SYNTAX_ERROR; extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; extern const int ILLEGAL_MYSQL_VARIABLE; } @@ -357,6 +358,114 @@ static inline void fillSignAndVersionColumnsData(Block & data, Int8 sign_value, data.getByPosition(data.columns() - 1).column = std::move(version_mutable_column); } +template +static void writeFieldsToColumn( + IColumn & column_to, const std::vector & rows_data, size_t column_index, const std::vector & mask, ColumnUInt8 * null_map_column = nullptr) +{ + if (ColumnNullable * column_nullable = typeid_cast(&column_to)) + writeFieldsToColumn(column_nullable->getNestedColumn(), rows_data, column_index, mask, &column_nullable->getNullMapColumn()); + else + { + const auto & write_data_to_null_map = [&](const Field & field, size_t row_index) + { + if (!mask.empty() && !mask.at(row_index)) + return false; + + if constexpr (assert_nullable) + { + if (field.isNull()) + { + column_to.insertDefault(); + null_map_column->insertDefault(); + return false; + } + + null_map_column->insertValue(0); + } + + return true; + }; + + const auto & write_data_to_column = [&](auto * casted_column, auto from_type, auto to_type) { + for (size_t index = 0; index < rows_data.size(); ++index) + { + const Field & value = DB::get(rows_data[index])[column_index]; + + if (write_data_to_null_map(value, index)) + casted_column->insertValue(static_cast(value.template get())); + } + }; + + if (ColumnInt8 * casted_column = typeid_cast(&column_to)) + write_data_to_column(casted_column, UInt8(), Int8()); + else if (ColumnInt16 * casted_column = typeid_cast(&column_to)) + write_data_to_column(casted_column, UInt16(), Int16()); + else if (ColumnInt64 * casted_column = typeid_cast(&column_to)) + write_data_to_column(casted_column, UInt64(), Int64()); + else if (ColumnUInt8 * casted_column = typeid_cast(&column_to)) + write_data_to_column(casted_column, UInt8(), UInt8()); + else if (ColumnUInt16 * casted_column = typeid_cast(&column_to)) + write_data_to_column(casted_column, UInt16(), UInt16()); + else if (ColumnUInt32 * casted_column = typeid_cast(&column_to)) + write_data_to_column(casted_column, UInt32(), UInt32()); + else if (ColumnUInt64 * casted_column = typeid_cast(&column_to)) + write_data_to_column(casted_column, UInt64(), UInt64()); + else if (ColumnFloat32 * casted_column = typeid_cast(&column_to)) + write_data_to_column(casted_column, Float32(), Float32()); + else if (ColumnFloat64 * casted_column = typeid_cast(&column_to)) + write_data_to_column(casted_column, Float64(), Float64()); + else if (ColumnInt32 * casted_column = typeid_cast(&column_to)) + { + for (size_t index = 0; index < rows_data.size(); ++index) + { + const Field & value = DB::get(rows_data[index])[column_index]; + + if (write_data_to_null_map(value, index)) + { + if (value.getType() == Field::Types::UInt64) + casted_column->insertValue(value.get()); + else if (value.getType() == Field::Types::Int64) + { + /// For MYSQL_TYPE_INT24 + const Int32 & num = value.get(); + casted_column->insertValue(num & 0x800000 ? num | 0xFF000000 : num); + } + else + throw Exception("LOGICAL ERROR: it is a bug.", ErrorCodes::LOGICAL_ERROR); + } + } + } + else if (ColumnString * casted_column = typeid_cast(&column_to)) + { + for (size_t index = 0; index < rows_data.size(); ++index) + { + const Field & value = DB::get(rows_data[index])[column_index]; + + if (write_data_to_null_map(value, index)) + { + const String & data = value.get(); + casted_column->insertData(data.data(), data.size()); + } + } + } + else if (ColumnFixedString * casted_column = typeid_cast(&column_to)) + { + for (size_t index = 0; index < rows_data.size(); ++index) + { + const Field & value = DB::get(rows_data[index])[column_index]; + + if (write_data_to_null_map(value, index)) + { + const String & data = value.get(); + casted_column->insertData(data.data(), data.size()); + } + } + } + else + throw Exception("Unsupported data type from MySQL.", ErrorCodes::NOT_IMPLEMENTED); + } +} + template static size_t onWriteOrDeleteData(const std::vector & rows_data, Block & buffer, size_t version) { @@ -365,9 +474,7 @@ static size_t onWriteOrDeleteData(const std::vector & rows_data, Block & { MutableColumnPtr col_to = IColumn::mutate(std::move(buffer.getByPosition(column).column)); - for (size_t index = 0; index < rows_data.size(); ++index) - col_to->insert(DB::get(rows_data[index])[column]); - + writeFieldsToColumn(*col_to, rows_data, column, {}); buffer.getByPosition(column).column = std::move(col_to); } @@ -387,31 +494,23 @@ static inline bool differenceSortingKeys(const Tuple & row_old_data, const Tuple static inline size_t onUpdateData(const std::vector & rows_data, Block & buffer, size_t version, const std::vector & sorting_columns_index) { if (rows_data.size() % 2 != 0) - throw Exception("LOGICAL ERROR: ", ErrorCodes::LOGICAL_ERROR); + throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR); size_t prev_bytes = buffer.bytes(); - std::vector difference_sorting_keys_mark(rows_data.size() / 2); + std::vector writeable_rows_mask(rows_data.size()); for (size_t index = 0; index < rows_data.size(); index += 2) - difference_sorting_keys_mark[index / 2] = differenceSortingKeys( + { + writeable_rows_mask[index + 1] = true; + writeable_rows_mask[index] = differenceSortingKeys( DB::get(rows_data[index]), DB::get(rows_data[index + 1]), sorting_columns_index); + } for (size_t column = 0; column < buffer.columns() - 2; ++column) { MutableColumnPtr col_to = IColumn::mutate(std::move(buffer.getByPosition(column).column)); - for (size_t index = 0; index < rows_data.size(); index += 2) - { - if (likely(!difference_sorting_keys_mark[index / 2])) - col_to->insert(DB::get(rows_data[index + 1])[column]); - else - { - /// If the sorting keys is modified, we should cancel the old data, but this should not happen frequently - col_to->insert(DB::get(rows_data[index])[column]); - col_to->insert(DB::get(rows_data[index + 1])[column]); - } - } - + writeFieldsToColumn(*col_to, rows_data, column, writeable_rows_mask); buffer.getByPosition(column).column = std::move(col_to); } @@ -423,7 +522,7 @@ static inline size_t onUpdateData(const std::vector & rows_data, Block & for (size_t index = 0; index < rows_data.size(); index += 2) { - if (likely(!difference_sorting_keys_mark[index / 2])) + if (likely(!writeable_rows_mask[index])) { sign_column_data.emplace_back(1); version_column_data.emplace_back(version); diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index d0cf5f2bedb..96a2bb6e63a 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -47,15 +47,36 @@ static inline NamesAndTypesList getColumnsList(ASTExpressionList * columns_defin throw Exception("Missing type in definition of column.", ErrorCodes::UNKNOWN_TYPE); bool is_nullable = true; + bool is_unsigned = false; if (declare_column->column_options) { - if (const auto * options = declare_column->column_options->as(); - options && options->changes.count("is_null")) - is_nullable = options->changes.at("is_null")->as()->value.safeGet(); + if (const auto * options = declare_column->column_options->as()) + { + if (options->changes.count("is_null")) + is_nullable = options->changes.at("is_null")->as()->value.safeGet(); + + if (options->changes.count("is_unsigned")) + is_unsigned = options->changes.at("is_unsigned")->as()->value.safeGet(); + } } ASTPtr data_type = declare_column->data_type; + if (is_unsigned) + { + auto data_type_function = data_type->as(); + + if (data_type_function) + { + String type_name_upper = Poco::toUpper(data_type_function->name); + + /// For example(in MySQL): CREATE TABLE test(column_name INT NOT NULL ... UNSIGNED) + if (type_name_upper.find("INT") != std::string::npos && !endsWith(type_name_upper, "SIGNED") + && !endsWith(type_name_upper, "UNSIGNED")) + data_type_function->name = type_name_upper + " UNSIGNED"; + } + } + if (is_nullable) data_type = makeASTFunction("Nullable", data_type); @@ -72,7 +93,7 @@ static NamesAndTypesList getNames(const ASTFunction & expr, const Context & cont ASTPtr temp_ast = expr.clone(); auto syntax = SyntaxAnalyzer(context).analyze(temp_ast, columns); - auto expression = ExpressionAnalyzer(temp_ast, syntax, context).getActions(true); + auto expression = ExpressionAnalyzer(temp_ast, syntax, context).getActions(false); return expression->getRequiredColumnsWithTypes(); } @@ -160,43 +181,68 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys) if (type->isNullable()) column = makeASTFunction("assumeNotNull", column); - return makeASTFunction("divide", column, std::make_shared(UInt64(type_max_size / 1000))); + return makeASTFunction("intDiv", column, std::make_shared(UInt64(type_max_size / 1000))); }; + ASTPtr best_partition; + size_t index = 0, best_size = 0; for (const auto & primary_key : primary_keys) { - WhichDataType which(primary_key.type); + DataTypePtr type = primary_key.type; + WhichDataType which(type); if (which.isNullable()) - which = WhichDataType((static_cast(*primary_key.type)).getNestedType()); + { + type = (static_cast(*type)).getNestedType(); + which = WhichDataType(type); + } if (which.isDateOrDateTime()) { + /// In any case, date or datetime is always the best partitioning key ASTPtr res = std::make_shared(primary_key.name); return makeASTFunction("toYYYYMM", primary_key.type->isNullable() ? makeASTFunction("assumeNotNull", res) : res); } - if (which.isInt8() || which.isUInt8()) - return std::make_shared(primary_key.name); - else if (which.isInt16() || which.isUInt16()) - return numbers_partition(primary_key.name, primary_key.type, std::numeric_limits::max()); - else if (which.isInt32() || which.isUInt32()) - return numbers_partition(primary_key.name, primary_key.type, std::numeric_limits::max()); - else if (which.isInt64() || which.isUInt64()) - return numbers_partition(primary_key.name, primary_key.type, std::numeric_limits::max()); + if (type->haveMaximumSizeOfValue() && (!best_size || type->getSizeOfValueInMemory() < best_size)) + { + if (which.isInt8() || which.isUInt8()) + { + best_size = type->getSizeOfValueInMemory(); + best_partition = std::make_shared(primary_key.name); + } + else if (which.isInt16() || which.isUInt16()) + { + best_size = type->getSizeOfValueInMemory(); + best_partition = numbers_partition(primary_key.name, type, std::numeric_limits::max()); + } + else if (which.isInt32() || which.isUInt32()) + { + best_size = type->getSizeOfValueInMemory(); + best_partition = numbers_partition(primary_key.name, type, std::numeric_limits::max()); + } + else if (which.isInt64() || which.isUInt64()) + { + best_size = type->getSizeOfValueInMemory(); + best_partition = numbers_partition(primary_key.name, type, std::numeric_limits::max()); + } + } } - return {}; + return best_partition; } static ASTPtr getOrderByPolicy( const NamesAndTypesList & primary_keys, const NamesAndTypesList & unique_keys, const NamesAndTypesList & keys, const NameSet & increment_columns) { NameSet order_by_columns_set; - std::deque order_by_columns; + std::deque> order_by_columns_list; const auto & add_order_by_expression = [&](const NamesAndTypesList & names_and_types) { + std::vector increment_keys; + std::vector non_increment_keys; + for (const auto & [name, type] : names_and_types) { if (order_by_columns_set.contains(name)) @@ -204,18 +250,21 @@ static ASTPtr getOrderByPolicy( if (increment_columns.contains(name)) { + increment_keys.emplace_back(name); order_by_columns_set.emplace(name); - order_by_columns.emplace_back(name); } else { order_by_columns_set.emplace(name); - order_by_columns.emplace_front(name); + non_increment_keys.emplace_back(name); } } + + order_by_columns_list.emplace_back(increment_keys); + order_by_columns_list.emplace_front(non_increment_keys); }; - /// primary_key[not increment], key[not increment], unique[not increment], key[increment], unique[increment], primary_key[increment] + /// primary_key[not increment], key[not increment], unique[not increment], unique[increment], key[increment], primary_key[increment] add_order_by_expression(unique_keys); add_order_by_expression(keys); add_order_by_expression(primary_keys); @@ -224,8 +273,11 @@ static ASTPtr getOrderByPolicy( order_by_expression->name = "tuple"; order_by_expression->arguments = std::make_shared(); - for (const auto & order_by_column : order_by_columns) - order_by_expression->arguments->children.emplace_back(std::make_shared(order_by_column)); + for (const auto & order_by_columns : order_by_columns_list) + { + for (const auto & order_by_column : order_by_columns) + order_by_expression->arguments->children.emplace_back(std::make_shared(order_by_column)); + } return order_by_expression; } diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp new file mode 100644 index 00000000000..1a1568c0863 --- /dev/null +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -0,0 +1,59 @@ +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; + +static inline ASTPtr tryRewrittenCreateQuery(const String & query, Context & context) +{ + ParserExternalDDLQuery external_ddl_parser; + ASTPtr ast = parseQuery(external_ddl_parser, query, 0, 0); + + context.unsafeSetCurrentDatabase("default"); + return MySQLInterpreter::InterpreterCreateImpl::getRewrittenQuery( + *ast->as()->external_ddl->as(), + context, "test_database", "test_database"); +} + +TEST(MySQLCreateRewritten, RewrittenQueryWithPrimaryKey) +{ + registerFunctions(); + auto shared_context = Context::createShared(); + auto global_context = std::make_unique(Context::createGlobal(shared_context.get())); + + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL PRIMARY " + "KEY) ENGINE=InnoDB DEFAULT CHARSET=utf8", *global_context)), + "CREATE TABLE test_database.test_table_1 (`key` Int32, `_sign` Int8, `_version` UInt64) ENGINE = ReplacingMergeTree(_version) " + "PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); + + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL, " + " PRIMARY KEY (`key`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", *global_context)), + "CREATE TABLE test_database.test_table_1 (`key` Int32, `_sign` Int8, `_version` UInt64) ENGINE = ReplacingMergeTree(_version) " + "PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); + + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key_1` int NOT NULL, " + " key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", *global_context)), + "CREATE TABLE test_database.test_table_1 (`key_1` Int32, `key_2` Int32, `_sign` Int8, `_version` UInt64) ENGINE = " + "ReplacingMergeTree(_version) PARTITION BY intDiv(key_1, 4294967) ORDER BY (key_1, key_2)"); + + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key_1` BIGINT NOT NULL, " + " key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", *global_context)), + "CREATE TABLE test_database.test_table_1 (`key_1` Int64, `key_2` Int32, `_sign` Int8, `_version` UInt64) ENGINE = " + "ReplacingMergeTree(_version) PARTITION BY intDiv(key_2, 4294967) ORDER BY (key_1, key_2)"); +} + diff --git a/src/Parsers/MySQL/ASTDeclareColumn.cpp b/src/Parsers/MySQL/ASTDeclareColumn.cpp index 66f49aed565..41744ad5fa4 100644 --- a/src/Parsers/MySQL/ASTDeclareColumn.cpp +++ b/src/Parsers/MySQL/ASTDeclareColumn.cpp @@ -70,6 +70,8 @@ bool ParserDeclareColumn::parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & ParserDeclareOptions p_non_generate_options{ { OptionDescribe("ZEROFILL", "zero_fill", std::make_unique()), + OptionDescribe("SIGNED", "is_unsigned", std::make_unique()), + OptionDescribe("UNSIGNED", "is_unsigned", std::make_unique()), OptionDescribe("NULL", "is_null", std::make_unique()), OptionDescribe("NOT NULL", "is_null", std::make_unique()), OptionDescribe("DEFAULT", "default", std::make_unique()), diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 1dc5827e2b1..a1a8920981a 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -15,6 +15,74 @@ def check_query(clickhouse_node, query, result_set, retry_count=3, interval_seco assert lastest_result == result_set +def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + # existed before the mapping was created + # TODO: Add check test BIT[(M)] BOOL, BOOLEAN, Enum + + mysql_node.query("CREATE TABLE test_database.test_table_1 (" + "`key` INT NOT NULL PRIMARY KEY, " + "unsigned_tiny_int TINYINT UNSIGNED, tiny_int TINYINT, " + "unsigned_small_int SMALLINT UNSIGNED, small_int SMALLINT, " + "unsigned_medium_int MEDIUMINT UNSIGNED, medium_int MEDIUMINT, " + "unsigned_int INT UNSIGNED, _int INT, " + "unsigned_integer INTEGER UNSIGNED, _integer INTEGER, " + "unsigned_bigint BIGINT UNSIGNED, _bigint BIGINT, " + "/* Need ClickHouse support read mysql decimal unsigned_decimal DECIMAL(19, 10) UNSIGNED, _decimal DECIMAL(19, 10), */" + "unsigned_float FLOAT UNSIGNED, _float FLOAT, " + "unsigned_double DOUBLE UNSIGNED, _double DOUBLE, " + "_varchar VARCHAR(10), _char CHAR(10), " + "_date Date, _datetime DateTime, _timestamp TIMESTAMP) ENGINE = InnoDB;") + + # it already has some data + mysql_node.query( + "INSERT INTO test_database.test_table_1 VALUES(1, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', " + "'2020-01-01', '2020-01-01 00:00:00', '2020-01-01 00:00:00');") + + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) + + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" + "2020-01-01 00:00:00\t2020-01-01 00:00:00\n") + + mysql_node.query( + "INSERT INTO test_database.test_table_1 VALUES(2, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', " + "'2020-01-01', '2020-01-01 00:00:00', '2020-01-01 00:00:00');") + + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" + "2020-01-01 00:00:00\t2020-01-01 00:00:00\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\t" + "varchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\n") + + mysql_node.query("UPDATE test_database.test_table_1 SET unsigned_tiny_int = 2 WHERE `key` = 1") + + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + "1\t2\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" + "2020-01-01 00:00:00\t2020-01-01 00:00:00\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\t" + "varchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\n") + + # update primary key + mysql_node.query("UPDATE test_database.test_table_1 SET `key` = 3 WHERE `tiny_int` = -1") + + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + "2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\t" + "varchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\n3\t2\t-1\t2\t-2\t3\t-3\t" + "4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\n") + + mysql_node.query('DELETE FROM test_database.test_table_1 WHERE `key` = 2') + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + "3\t2\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" + "2020-01-01 00:00:00\t2020-01-01 00:00:00\n") + + mysql_node.query('DELETE FROM test_database.test_table_1 WHERE `unsigned_tiny_int` = 2') + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", "") + + mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database") + + def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 03b2d15d404..6100d0bba37 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -87,6 +87,14 @@ def started_mysql_8_0(): subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes', '--remove-orphans']) +def test_materialize_database_dml_with_mysql_5_7(started_cluster, started_mysql_5_7): + materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + + +def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_8_0): + materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + + def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7): materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") @@ -106,3 +114,4 @@ def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_ materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + From c0db408c3868373e321b9002f35ef4d62c238dd8 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 21 Jul 2020 02:33:28 +0800 Subject: [PATCH 111/374] ISSUES-4006 try fix build failure --- src/Databases/MySQL/MaterializeMySQLSyncThread.cpp | 3 ++- src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index e1f07fe7373..6a036f9adab 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -386,7 +386,8 @@ static void writeFieldsToColumn( return true; }; - const auto & write_data_to_column = [&](auto * casted_column, auto from_type, auto to_type) { + const auto & write_data_to_column = [&](auto * casted_column, auto from_type, auto to_type) + { for (size_t index = 0; index < rows_data.size(); ++index) { const Field & value = DB::get(rows_data[index])[column_index]; diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 96a2bb6e63a..b86598f927e 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -185,7 +185,7 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys) }; ASTPtr best_partition; - size_t index = 0, best_size = 0; + size_t best_size = 0; for (const auto & primary_key : primary_keys) { DataTypePtr type = primary_key.type; From 6b452cf83d0f7234a6a1bcf3ddf1fc6b7b766b7e Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 21 Jul 2020 09:35:25 +0800 Subject: [PATCH 112/374] ISSUES-4006 try fix build failure & fix float 32 parser --- .../tests/memory_statistics_os_perf.cpp | 4 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 53 ++++++++++--------- .../materialize_with_ddl.py | 34 ++++++------ 3 files changed, 47 insertions(+), 44 deletions(-) diff --git a/src/Common/tests/memory_statistics_os_perf.cpp b/src/Common/tests/memory_statistics_os_perf.cpp index 61ca81861d5..89c513a43dd 100644 --- a/src/Common/tests/memory_statistics_os_perf.cpp +++ b/src/Common/tests/memory_statistics_os_perf.cpp @@ -1,8 +1,10 @@ +#if defined(OS_LINUX) #include #include +#endif -int main(int argc, char ** argv) +int main([[maybe_unused]] int argc, [[maybe_unused]] char ** argv) { #if defined(OS_LINUX) using namespace DB; diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 6a036f9adab..c142cbc43dc 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -6,8 +6,8 @@ #include -# include # include +# include # include # include # include @@ -22,6 +22,7 @@ # include # include # include +# include namespace DB { @@ -397,25 +398,25 @@ static void writeFieldsToColumn( } }; - if (ColumnInt8 * casted_column = typeid_cast(&column_to)) - write_data_to_column(casted_column, UInt8(), Int8()); - else if (ColumnInt16 * casted_column = typeid_cast(&column_to)) - write_data_to_column(casted_column, UInt16(), Int16()); - else if (ColumnInt64 * casted_column = typeid_cast(&column_to)) - write_data_to_column(casted_column, UInt64(), Int64()); - else if (ColumnUInt8 * casted_column = typeid_cast(&column_to)) - write_data_to_column(casted_column, UInt8(), UInt8()); - else if (ColumnUInt16 * casted_column = typeid_cast(&column_to)) - write_data_to_column(casted_column, UInt16(), UInt16()); - else if (ColumnUInt32 * casted_column = typeid_cast(&column_to)) - write_data_to_column(casted_column, UInt32(), UInt32()); - else if (ColumnUInt64 * casted_column = typeid_cast(&column_to)) - write_data_to_column(casted_column, UInt64(), UInt64()); - else if (ColumnFloat32 * casted_column = typeid_cast(&column_to)) - write_data_to_column(casted_column, Float32(), Float32()); - else if (ColumnFloat64 * casted_column = typeid_cast(&column_to)) - write_data_to_column(casted_column, Float64(), Float64()); - else if (ColumnInt32 * casted_column = typeid_cast(&column_to)) + if (ColumnInt8 * casted_int8_column = typeid_cast(&column_to)) + write_data_to_column(casted_int8_column, UInt64(), Int8()); + else if (ColumnInt16 * casted_int16_column = typeid_cast(&column_to)) + write_data_to_column(casted_int16_column, UInt64(), Int16()); + else if (ColumnInt64 * casted_int64_column = typeid_cast(&column_to)) + write_data_to_column(casted_int64_column, UInt64(), Int64()); + else if (ColumnUInt8 * casted_uint8_column = typeid_cast(&column_to)) + write_data_to_column(casted_uint8_column, UInt64(), UInt8()); + else if (ColumnUInt16 * casted_uint16_column = typeid_cast(&column_to)) + write_data_to_column(casted_uint16_column, UInt64(), UInt16()); + else if (ColumnUInt32 * casted_uint32_column = typeid_cast(&column_to)) + write_data_to_column(casted_uint32_column, UInt64(), UInt32()); + else if (ColumnUInt64 * casted_uint64_column = typeid_cast(&column_to)) + write_data_to_column(casted_uint64_column, UInt64(), UInt64()); + else if (ColumnFloat32 * casted_float32_column = typeid_cast(&column_to)) + write_data_to_column(casted_float32_column, Float64(), Float32()); + else if (ColumnFloat64 * casted_float64_column = typeid_cast(&column_to)) + write_data_to_column(casted_float64_column, Float64(), Float64()); + else if (ColumnInt32 * casted_int32_column = typeid_cast(&column_to)) { for (size_t index = 0; index < rows_data.size(); ++index) { @@ -424,19 +425,19 @@ static void writeFieldsToColumn( if (write_data_to_null_map(value, index)) { if (value.getType() == Field::Types::UInt64) - casted_column->insertValue(value.get()); + casted_int32_column->insertValue(value.get()); else if (value.getType() == Field::Types::Int64) { /// For MYSQL_TYPE_INT24 const Int32 & num = value.get(); - casted_column->insertValue(num & 0x800000 ? num | 0xFF000000 : num); + casted_int32_column->insertValue(num & 0x800000 ? num | 0xFF000000 : num); } else throw Exception("LOGICAL ERROR: it is a bug.", ErrorCodes::LOGICAL_ERROR); } } } - else if (ColumnString * casted_column = typeid_cast(&column_to)) + else if (ColumnString * casted_string_column = typeid_cast(&column_to)) { for (size_t index = 0; index < rows_data.size(); ++index) { @@ -445,11 +446,11 @@ static void writeFieldsToColumn( if (write_data_to_null_map(value, index)) { const String & data = value.get(); - casted_column->insertData(data.data(), data.size()); + casted_string_column->insertData(data.data(), data.size()); } } } - else if (ColumnFixedString * casted_column = typeid_cast(&column_to)) + else if (ColumnFixedString * casted_fixed_string_column = typeid_cast(&column_to)) { for (size_t index = 0; index < rows_data.size(); ++index) { @@ -458,7 +459,7 @@ static void writeFieldsToColumn( if (write_data_to_null_map(value, index)) { const String & data = value.get(); - casted_column->insertData(data.data(), data.size()); + casted_fixed_string_column->insertData(data.data(), data.size()); } } } diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index a1a8920981a..02cbf53c072 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -18,7 +18,6 @@ def check_query(clickhouse_node, query, result_set, retry_count=3, interval_seco def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") # existed before the mapping was created - # TODO: Add check test BIT[(M)] BOOL, BOOLEAN, Enum mysql_node.query("CREATE TABLE test_database.test_table_1 (" "`key` INT NOT NULL PRIMARY KEY, " @@ -32,49 +31,50 @@ def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_nam "unsigned_float FLOAT UNSIGNED, _float FLOAT, " "unsigned_double DOUBLE UNSIGNED, _double DOUBLE, " "_varchar VARCHAR(10), _char CHAR(10), " - "_date Date, _datetime DateTime, _timestamp TIMESTAMP) ENGINE = InnoDB;") + "/* Need ClickHouse support Enum('a', 'b', 'v') _enum ENUM('a', 'b', 'c'), */" + "_date Date, _datetime DateTime, _timestamp TIMESTAMP, _bool BOOLEAN) ENGINE = InnoDB;") # it already has some data mysql_node.query( "INSERT INTO test_database.test_table_1 VALUES(1, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', " - "'2020-01-01', '2020-01-01 00:00:00', '2020-01-01 00:00:00');") + "'2020-01-01', '2020-01-01 00:00:00', '2020-01-01 00:00:00', true);") clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", - "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" - "2020-01-01 00:00:00\t2020-01-01 00:00:00\n") + "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" + "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n") mysql_node.query( "INSERT INTO test_database.test_table_1 VALUES(2, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', " - "'2020-01-01', '2020-01-01 00:00:00', '2020-01-01 00:00:00');") + "'2020-01-01', '2020-01-01 00:00:00', '2020-01-01 00:00:00', false);") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", - "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" - "2020-01-01 00:00:00\t2020-01-01 00:00:00\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\t" - "varchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\n") + "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" + "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\t" + "varchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\t0\n") mysql_node.query("UPDATE test_database.test_table_1 SET unsigned_tiny_int = 2 WHERE `key` = 1") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", - "1\t2\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" - "2020-01-01 00:00:00\t2020-01-01 00:00:00\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\t" - "varchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\n") + "1\t2\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" + "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\t" + "varchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\t0\n") # update primary key mysql_node.query("UPDATE test_database.test_table_1 SET `key` = 3 WHERE `tiny_int` = -1") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", - "2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\t" - "varchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\n3\t2\t-1\t2\t-2\t3\t-3\t" - "4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\n") + "2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\t" + "varchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\t0\n3\t2\t-1\t2\t-2\t3\t-3\t" + "4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n") mysql_node.query('DELETE FROM test_database.test_table_1 WHERE `key` = 2') check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", - "3\t2\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" - "2020-01-01 00:00:00\t2020-01-01 00:00:00\n") + "3\t2\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" + "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n") mysql_node.query('DELETE FROM test_database.test_table_1 WHERE `unsigned_tiny_int` = 2') check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", "") From c173bd391fb316ae2b449c720c20af1b884cabd3 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 21 Jul 2020 16:16:15 +0800 Subject: [PATCH 113/374] ISSUES-4006 try fix build & test failure --- src/Common/tests/CMakeLists.txt | 6 +- src/Core/MySQLReplication.cpp | 1 + src/DataTypes/DataTypesNumber.cpp | 8 +- src/Databases/MySQL/MaterializeMetadata.cpp | 2 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 16 +- .../MySQL/MaterializeMySQLSyncThread.h | 2 +- .../MySQL/InterpretersMySQLDDLQuery.cpp | 2 +- src/Parsers/MySQL/ASTAlterCommand.cpp | 64 +++--- src/Parsers/MySQL/ASTAlterCommand.h | 22 +- src/Parsers/MySQL/ASTDeclareColumn.cpp | 64 +++--- src/Parsers/MySQL/ASTDeclareColumn.h | 2 - src/Parsers/MySQL/ASTDeclareIndex.cpp | 213 +++++++++--------- src/Parsers/MySQL/ASTDeclareIndex.h | 5 - .../MySQL/ASTDeclarePartitionOptions.cpp | 124 +++++----- .../MySQL/ASTDeclarePartitionOptions.h | 2 - .../composes/mysql_5_7_compose.yml | 2 +- .../composes/mysql_8_0_compose.yml | 2 +- .../materialize_with_ddl.py | 34 ++- 18 files changed, 287 insertions(+), 284 deletions(-) diff --git a/src/Common/tests/CMakeLists.txt b/src/Common/tests/CMakeLists.txt index 2653ab30c29..8df5c9c39ed 100644 --- a/src/Common/tests/CMakeLists.txt +++ b/src/Common/tests/CMakeLists.txt @@ -74,8 +74,10 @@ target_link_libraries (symbol_index PRIVATE clickhouse_common_io) add_executable (chaos_sanitizer chaos_sanitizer.cpp) target_link_libraries (chaos_sanitizer PRIVATE clickhouse_common_io) -add_executable (memory_statistics_os_perf memory_statistics_os_perf.cpp) -target_link_libraries (memory_statistics_os_perf PRIVATE clickhouse_common_io) +if (OS_LINUX) + add_executable (memory_statistics_os_perf memory_statistics_os_perf.cpp) + target_link_libraries (memory_statistics_os_perf PRIVATE clickhouse_common_io) +endif() add_executable (procfs_metrics_provider_perf procfs_metrics_provider_perf.cpp) target_link_libraries (procfs_metrics_provider_perf PRIVATE clickhouse_common_io) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index f7a817f8124..1933a11b036 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -398,6 +398,7 @@ namespace MySQLReplication } case MYSQL_TYPE_TIMESTAMP: { UInt32 val = 0; + payload.readStrict(reinterpret_cast(&val), 4); row.push_back(Field{val}); break; diff --git a/src/DataTypes/DataTypesNumber.cpp b/src/DataTypes/DataTypesNumber.cpp index 883dafdf5b0..8dd1d3a3f8c 100644 --- a/src/DataTypes/DataTypesNumber.cpp +++ b/src/DataTypes/DataTypesNumber.cpp @@ -35,10 +35,10 @@ static DataTypePtr createNumericDataType(const ASTPtr & arguments) void registerDataTypeNumbers(DataTypeFactory & factory) { - factory.registerSimpleDataType("UInt8", [] { return DataTypePtr(std::make_shared()); }); - factory.registerSimpleDataType("UInt16", [] { return DataTypePtr(std::make_shared()); }); - factory.registerSimpleDataType("UInt32", [] { return DataTypePtr(std::make_shared()); }); - factory.registerSimpleDataType("UInt64", [] { return DataTypePtr(std::make_shared()); }); + factory.registerDataType("UInt8", createNumericDataType); + factory.registerDataType("UInt16", createNumericDataType); + factory.registerDataType("UInt32", createNumericDataType); + factory.registerDataType("UInt64", createNumericDataType); factory.registerDataType("Int8", createNumericDataType); factory.registerDataType("Int16", createNumericDataType); diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 87540648876..0b470dda903 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -155,7 +155,7 @@ void MaterializeMetadata::transaction(const MySQLReplication::Position & positio out.close(); } - commitMetadata(fun, persistent_tmp_path, persistent_path); + commitMetadata(std::move(fun), persistent_tmp_path, persistent_path); } MaterializeMetadata::MaterializeMetadata( diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index c142cbc43dc..fc105ce177d 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -377,7 +377,7 @@ static void writeFieldsToColumn( if (field.isNull()) { column_to.insertDefault(); - null_map_column->insertDefault(); + null_map_column->insertValue(1); return false; } @@ -391,7 +391,8 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Field & value = DB::get(rows_data[index])[column_index]; + const Tuple & row_data = DB::get(rows_data[index]); + const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) casted_column->insertValue(static_cast(value.template get())); @@ -420,7 +421,8 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Field & value = DB::get(rows_data[index])[column_index]; + const Tuple & row_data = DB::get(rows_data[index]); + const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) { @@ -441,7 +443,8 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Field & value = DB::get(rows_data[index])[column_index]; + const Tuple & row_data = DB::get(rows_data[index]); + const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) { @@ -454,7 +457,8 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Field & value = DB::get(rows_data[index])[column_index]; + const Tuple & row_data = DB::get(rows_data[index]); + const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) { @@ -600,7 +604,7 @@ void MaterializeMySQLSyncThread::Buffers::add(size_t block_rows, size_t block_by max_block_bytes = std::max(block_bytes, max_block_bytes); } -bool MaterializeMySQLSyncThread::Buffers::checkThresholds(size_t check_block_rows, size_t check_block_bytes, size_t check_total_rows, size_t check_total_bytes) +bool MaterializeMySQLSyncThread::Buffers::checkThresholds(size_t check_block_rows, size_t check_block_bytes, size_t check_total_rows, size_t check_total_bytes) const { return max_block_rows >= check_block_rows || max_block_bytes >= check_block_bytes || total_blocks_rows >= check_total_rows || total_blocks_bytes >= check_total_bytes; diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index 18a0bba7a25..4d4dbc28624 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -68,7 +68,7 @@ private: void add(size_t block_rows, size_t block_bytes, size_t written_rows, size_t written_bytes); - bool checkThresholds(size_t check_block_rows, size_t check_block_bytes, size_t check_total_rows, size_t check_total_bytes); + bool checkThresholds(size_t check_block_rows, size_t check_block_bytes, size_t check_total_rows, size_t check_total_bytes) const; BufferAndSortingColumnsPtr getTableDataBuffer(const String & table, const Context & context); }; diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index b86598f927e..7e5008569de 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -64,7 +64,7 @@ static inline NamesAndTypesList getColumnsList(ASTExpressionList * columns_defin if (is_unsigned) { - auto data_type_function = data_type->as(); + auto * data_type_function = data_type->as(); if (data_type_function) { diff --git a/src/Parsers/MySQL/ASTAlterCommand.cpp b/src/Parsers/MySQL/ASTAlterCommand.cpp index 8fa734c9997..c95aacb4e28 100644 --- a/src/Parsers/MySQL/ASTAlterCommand.cpp +++ b/src/Parsers/MySQL/ASTAlterCommand.cpp @@ -29,7 +29,7 @@ ASTPtr ASTAlterCommand::clone() const res->set(res->additional_columns, additional_columns->clone()); if (order_by_columns) - res->set(res->order_by_columns, additional_columns->clone()); + res->set(res->order_by_columns, order_by_columns->clone()); if (properties) res->set(res->properties, properties->clone()); @@ -37,32 +37,7 @@ ASTPtr ASTAlterCommand::clone() const return res; } -bool ParserAlterCommand::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserKeyword k_add("ADD"); - ParserKeyword k_drop("DROP"); - ParserKeyword k_alter("ALTER"); - ParserKeyword k_rename("RENAME"); - ParserKeyword k_modify("MODIFY"); - ParserKeyword k_change("CHANGE"); - - if (k_add.ignore(pos, expected)) - return parseAddCommand(pos, node, expected); - else if (k_drop.ignore(pos, expected)) - return parseDropCommand(pos, node, expected); - else if (k_alter.ignore(pos, expected)) - return parseAlterCommand(pos, node, expected); - else if (k_rename.ignore(pos, expected)) - return parseRenameCommand(pos, node, expected); - else if (k_modify.ignore(pos, expected)) - return parseModifyCommand(pos, node, expected); - else if (k_change.ignore(pos, expected)) - return parseModifyCommand(pos, node, expected, true); - else - return parseOtherCommand(pos, node, expected); -} - -bool ParserAlterCommand::parseAddCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) +static inline bool parseAddCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr declare_index; ASTPtr additional_columns; @@ -120,7 +95,7 @@ bool ParserAlterCommand::parseAddCommand(IParser::Pos & pos, ASTPtr & node, Expe return true; } -bool ParserAlterCommand::parseDropCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) +static inline bool parseDropCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr name; ParserIdentifier identifier_p; @@ -173,7 +148,7 @@ bool ParserAlterCommand::parseDropCommand(IParser::Pos & pos, ASTPtr & node, Exp return true; } -bool ParserAlterCommand::parseAlterCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) +static inline bool parseAlterCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr name; @@ -241,7 +216,7 @@ bool ParserAlterCommand::parseAlterCommand(IParser::Pos & pos, ASTPtr & node, Ex return true; } -bool ParserAlterCommand::parseRenameCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) +static inline bool parseRenameCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr old_name; ASTPtr new_name; @@ -296,7 +271,7 @@ bool ParserAlterCommand::parseRenameCommand(IParser::Pos & pos, ASTPtr & node, E return true; } -bool ParserAlterCommand::parseOtherCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) +static inline bool parseOtherCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected) { auto alter_command = std::make_shared(); @@ -346,7 +321,7 @@ bool ParserAlterCommand::parseOtherCommand(IParser::Pos & pos, ASTPtr & node, Ex return true; } -bool ParserAlterCommand::parseModifyCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected, bool exists_old_column_name) +static inline bool parseModifyCommand(IParser::Pos & pos, ASTPtr & node, Expected & expected, bool exists_old_column_name = false) { ASTPtr old_column_name; auto alter_command = std::make_shared(); @@ -381,6 +356,31 @@ bool ParserAlterCommand::parseModifyCommand(IParser::Pos & pos, ASTPtr & node, E return true; } + +bool ParserAlterCommand::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword k_add("ADD"); + ParserKeyword k_drop("DROP"); + ParserKeyword k_alter("ALTER"); + ParserKeyword k_rename("RENAME"); + ParserKeyword k_modify("MODIFY"); + ParserKeyword k_change("CHANGE"); + + if (k_add.ignore(pos, expected)) + return parseAddCommand(pos, node, expected); + else if (k_drop.ignore(pos, expected)) + return parseDropCommand(pos, node, expected); + else if (k_alter.ignore(pos, expected)) + return parseAlterCommand(pos, node, expected); + else if (k_rename.ignore(pos, expected)) + return parseRenameCommand(pos, node, expected); + else if (k_modify.ignore(pos, expected)) + return parseModifyCommand(pos, node, expected); + else if (k_change.ignore(pos, expected)) + return parseModifyCommand(pos, node, expected, true); + else + return parseOtherCommand(pos, node, expected); +} } } diff --git a/src/Parsers/MySQL/ASTAlterCommand.h b/src/Parsers/MySQL/ASTAlterCommand.h index 2e7fea705b1..3667116952b 100644 --- a/src/Parsers/MySQL/ASTAlterCommand.h +++ b/src/Parsers/MySQL/ASTAlterCommand.h @@ -43,15 +43,15 @@ public: Type type = NO_TYPE; /// For ADD INDEX - ASTDeclareIndex * index_decl; + ASTDeclareIndex * index_decl = nullptr; /// For modify default expression - IAST * default_expression; + IAST * default_expression = nullptr; /// For ADD COLUMN - ASTExpressionList * additional_columns; + ASTExpressionList * additional_columns = nullptr; /// For ORDER BY - ASTExpressionList * order_by_columns; + ASTExpressionList * order_by_columns = nullptr; bool first = false; bool index_visible = false; @@ -63,7 +63,7 @@ public: String column_name; String constraint_name; - IAST * properties; + IAST * properties = nullptr; ASTPtr clone() const override; @@ -76,18 +76,6 @@ protected: const char * getName() const override { return "alter command"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; - - bool parseAddCommand(Pos & pos, ASTPtr & node, Expected & expected); - - bool parseDropCommand(Pos & pos, ASTPtr & node, Expected & expected); - - bool parseAlterCommand(Pos & pos, ASTPtr & node, Expected & expected); - - bool parseRenameCommand(Pos & pos, ASTPtr & node, Expected & expected); - - bool parseModifyCommand(Pos & pos, ASTPtr & node, Expected & expected, bool exists_old_column_name = false); - - bool parseOtherCommand(Pos & pos, ASTPtr & node, Expected & expected); }; } diff --git a/src/Parsers/MySQL/ASTDeclareColumn.cpp b/src/Parsers/MySQL/ASTDeclareColumn.cpp index 41744ad5fa4..56a92291f06 100644 --- a/src/Parsers/MySQL/ASTDeclareColumn.cpp +++ b/src/Parsers/MySQL/ASTDeclareColumn.cpp @@ -34,6 +34,39 @@ ASTPtr ASTDeclareColumn::clone() const return res; } +static inline bool parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserDeclareOptions p_non_generate_options{ + { + OptionDescribe("ZEROFILL", "zero_fill", std::make_unique()), + OptionDescribe("SIGNED", "is_unsigned", std::make_unique()), + OptionDescribe("UNSIGNED", "is_unsigned", std::make_unique()), + OptionDescribe("NULL", "is_null", std::make_unique()), + OptionDescribe("NOT NULL", "is_null", std::make_unique()), + OptionDescribe("DEFAULT", "default", std::make_unique()), + OptionDescribe("ON UPDATE", "on_update", std::make_unique()), + OptionDescribe("AUTO_INCREMENT", "auto_increment", std::make_unique()), + OptionDescribe("UNIQUE", "unique_key", std::make_unique()), + OptionDescribe("UNIQUE KEY", "unique_key", std::make_unique()), + OptionDescribe("KEY", "primary_key", std::make_unique()), + OptionDescribe("PRIMARY KEY", "primary_key", std::make_unique()), + OptionDescribe("COMMENT", "comment", std::make_unique()), + OptionDescribe("CHARACTER SET", "charset_name", std::make_unique()), + OptionDescribe("COLLATE", "collate", std::make_unique()), + OptionDescribe("COLUMN_FORMAT", "column_format", std::make_unique()), + OptionDescribe("STORAGE", "storage", std::make_unique()), + OptionDescribe("AS", "generated", std::make_unique()), + OptionDescribe("GENERATED ALWAYS AS", "generated", std::make_unique()), + OptionDescribe("STORED", "is_stored", std::make_unique()), + OptionDescribe("VIRTUAL", "is_stored", std::make_unique()), + OptionDescribe("", "reference", std::make_unique()), + OptionDescribe("", "constraint", std::make_unique()), + } + }; + + return p_non_generate_options.parse(pos, node, expected); +} + bool ParserDeclareColumn::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr column_name; @@ -65,37 +98,6 @@ bool ParserDeclareColumn::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte node = declare_column; return true; } -bool ParserDeclareColumn::parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserDeclareOptions p_non_generate_options{ - { - OptionDescribe("ZEROFILL", "zero_fill", std::make_unique()), - OptionDescribe("SIGNED", "is_unsigned", std::make_unique()), - OptionDescribe("UNSIGNED", "is_unsigned", std::make_unique()), - OptionDescribe("NULL", "is_null", std::make_unique()), - OptionDescribe("NOT NULL", "is_null", std::make_unique()), - OptionDescribe("DEFAULT", "default", std::make_unique()), - OptionDescribe("AUTO_INCREMENT", "auto_increment", std::make_unique()), - OptionDescribe("UNIQUE", "unique_key", std::make_unique()), - OptionDescribe("UNIQUE KEY", "unique_key", std::make_unique()), - OptionDescribe("KEY", "primary_key", std::make_unique()), - OptionDescribe("PRIMARY KEY", "primary_key", std::make_unique()), - OptionDescribe("COMMENT", "comment", std::make_unique()), - OptionDescribe("CHARACTER SET", "charset_name", std::make_unique()), - OptionDescribe("COLLATE", "collate", std::make_unique()), - OptionDescribe("COLUMN_FORMAT", "column_format", std::make_unique()), - OptionDescribe("STORAGE", "storage", std::make_unique()), - OptionDescribe("AS", "generated", std::make_unique()), - OptionDescribe("GENERATED ALWAYS AS", "generated", std::make_unique()), - OptionDescribe("STORED", "is_stored", std::make_unique()), - OptionDescribe("VIRTUAL", "is_stored", std::make_unique()), - OptionDescribe("", "reference", std::make_unique()), - OptionDescribe("", "constraint", std::make_unique()), - } - }; - - return p_non_generate_options.parse(pos, node, expected); -} } diff --git a/src/Parsers/MySQL/ASTDeclareColumn.h b/src/Parsers/MySQL/ASTDeclareColumn.h index 25103072e71..75683c2ff4c 100644 --- a/src/Parsers/MySQL/ASTDeclareColumn.h +++ b/src/Parsers/MySQL/ASTDeclareColumn.h @@ -27,8 +27,6 @@ protected: const char * getName() const override { return "index declaration"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; - - bool parseColumnDeclareOptions(Pos & pos, ASTPtr & node, Expected & expected); }; diff --git a/src/Parsers/MySQL/ASTDeclareIndex.cpp b/src/Parsers/MySQL/ASTDeclareIndex.cpp index f6a1cf92857..0a6a2095261 100644 --- a/src/Parsers/MySQL/ASTDeclareIndex.cpp +++ b/src/Parsers/MySQL/ASTDeclareIndex.cpp @@ -57,6 +57,114 @@ ASTPtr ASTDeclareIndex::clone() const return res; } + +static inline bool parseDeclareOrdinaryIndex(IParser::Pos & pos, String & index_name, String & index_type, Expected & expected) +{ + ASTPtr temp_node; + ParserKeyword k_key("KEY"); + ParserKeyword k_index("INDEX"); + + ParserExpression p_expression; + ParserIdentifier p_identifier; + + if (ParserKeyword("SPATIAL").ignore(pos, expected)) + { + if (!k_key.ignore(pos, expected)) + k_index.ignore(pos, expected); + + index_type = "SPATIAL"; + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; + } + else if (ParserKeyword("FULLTEXT").ignore(pos, expected)) + { + if (!k_key.ignore(pos, expected)) + k_index.ignore(pos, expected); + + index_type = "FULLTEXT"; + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; + } + else + { + if (!k_key.ignore(pos, expected)) + { + if (!k_index.ignore(pos, expected)) + return false; + } + + index_type = "KEY_BTREE"; /// default index type + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; + + if (ParserKeyword("USING").ignore(pos, expected)) + { + if (!p_identifier.parse(pos, temp_node, expected)) + return false; + + index_type = "KEY_" + temp_node->as()->name; + } + } + + return true; +} + +static inline bool parseDeclareConstraintIndex(IParser::Pos & pos, String & index_name, String & index_type, Expected & expected) +{ + ASTPtr temp_node; + ParserIdentifier p_identifier; + + if (ParserKeyword("CONSTRAINT").ignore(pos, expected)) + { + + if (!ParserKeyword("PRIMARY").checkWithoutMoving(pos, expected) && !ParserKeyword("UNIQUE").checkWithoutMoving(pos, expected) + && !ParserKeyword("FOREIGN").checkWithoutMoving(pos, expected)) + { + if (!p_identifier.parse(pos, temp_node, expected)) + return false; + + index_name = temp_node->as()->name; + } + } + + if (ParserKeyword("UNIQUE").ignore(pos, expected)) + { + if (!ParserKeyword("KEY").ignore(pos, expected)) + ParserKeyword("INDEX").ignore(pos, expected); + + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; /// reset index_name + + index_type = "UNIQUE_BTREE"; /// default btree index_type + if (ParserKeyword("USING").ignore(pos, expected)) + { + if (!p_identifier.parse(pos, temp_node, expected)) + return false; + + index_type = "UNIQUE_" + temp_node->as()->name; + } + } + else if (ParserKeyword("PRIMARY KEY").ignore(pos, expected)) + { + index_type = "PRIMARY_KEY_BTREE"; /// default btree index_type + if (ParserKeyword("USING").ignore(pos, expected)) + { + if (!p_identifier.parse(pos, temp_node, expected)) + return false; + + index_type = "PRIMARY_KEY_" + temp_node->as()->name; + } + } + else if (ParserKeyword("FOREIGN KEY").ignore(pos, expected)) + { + index_type = "FOREIGN"; + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; /// reset index_name + } + + return true; +} + bool ParserDeclareIndex::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { String index_name; @@ -124,112 +232,7 @@ bool ParserDeclareIndex::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & node = declare_index; return true; } -bool ParserDeclareIndex::parseDeclareOrdinaryIndex(IParser::Pos & pos, String & index_name, String & index_type, Expected & expected) -{ - ASTPtr temp_node; - ParserKeyword k_key("KEY"); - ParserKeyword k_index("INDEX"); - ParserExpression p_expression; - ParserIdentifier p_identifier; - - if (ParserKeyword("SPATIAL").ignore(pos, expected)) - { - if (!k_key.ignore(pos, expected)) - k_index.ignore(pos, expected); - - index_type = "SPATIAL"; - if (p_identifier.parse(pos, temp_node, expected)) - index_name = temp_node->as()->name; - } - else if (ParserKeyword("FULLTEXT").ignore(pos, expected)) - { - if (!k_key.ignore(pos, expected)) - k_index.ignore(pos, expected); - - index_type = "FULLTEXT"; - if (p_identifier.parse(pos, temp_node, expected)) - index_name = temp_node->as()->name; - } - else - { - if (!k_key.ignore(pos, expected)) - { - if (!k_index.ignore(pos, expected)) - return false; - } - - index_type = "KEY_BTREE"; /// default index type - if (p_identifier.parse(pos, temp_node, expected)) - index_name = temp_node->as()->name; - - if (ParserKeyword("USING").ignore(pos, expected)) - { - if (!p_identifier.parse(pos, temp_node, expected)) - return false; - - index_type = "KEY_" + temp_node->as()->name; - } - } - - return true; -} - -bool ParserDeclareIndex::parseDeclareConstraintIndex(IParser::Pos & pos, String & index_name, String & index_type, Expected & expected) -{ - ASTPtr temp_node; - ParserIdentifier p_identifier; - - if (ParserKeyword("CONSTRAINT").ignore(pos, expected)) - { - - if (!ParserKeyword("PRIMARY").checkWithoutMoving(pos, expected) && !ParserKeyword("UNIQUE").checkWithoutMoving(pos, expected) - && !ParserKeyword("FOREIGN").checkWithoutMoving(pos, expected)) - { - if (!p_identifier.parse(pos, temp_node, expected)) - return false; - - index_name = temp_node->as()->name; - } - } - - if (ParserKeyword("UNIQUE").ignore(pos, expected)) - { - if (!ParserKeyword("KEY").ignore(pos, expected)) - ParserKeyword("INDEX").ignore(pos, expected); - - if (p_identifier.parse(pos, temp_node, expected)) - index_name = temp_node->as()->name; /// reset index_name - - index_type = "UNIQUE_BTREE"; /// default btree index_type - if (ParserKeyword("USING").ignore(pos, expected)) - { - if (!p_identifier.parse(pos, temp_node, expected)) - return false; - - index_type = "UNIQUE_" + temp_node->as()->name; - } - } - else if (ParserKeyword("PRIMARY KEY").ignore(pos, expected)) - { - index_type = "PRIMARY_KEY_BTREE"; /// default btree index_type - if (ParserKeyword("USING").ignore(pos, expected)) - { - if (!p_identifier.parse(pos, temp_node, expected)) - return false; - - index_type = "PRIMARY_KEY_" + temp_node->as()->name; - } - } - else if (ParserKeyword("FOREIGN KEY").ignore(pos, expected)) - { - index_type = "FOREIGN"; - if (p_identifier.parse(pos, temp_node, expected)) - index_name = temp_node->as()->name; /// reset index_name - } - - return true; -} } } diff --git a/src/Parsers/MySQL/ASTDeclareIndex.h b/src/Parsers/MySQL/ASTDeclareIndex.h index d3dc3741c0f..8488b5d2144 100644 --- a/src/Parsers/MySQL/ASTDeclareIndex.h +++ b/src/Parsers/MySQL/ASTDeclareIndex.h @@ -31,11 +31,6 @@ protected: const char * getName() const override { return "index declaration"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; - - bool parseDeclareOrdinaryIndex(Pos & pos, String & index_name, String & index_type, Expected & expected); - - bool parseDeclareConstraintIndex(Pos & pos, String & index_name, String & index_type, Expected & expected); - }; } diff --git a/src/Parsers/MySQL/ASTDeclarePartitionOptions.cpp b/src/Parsers/MySQL/ASTDeclarePartitionOptions.cpp index ff1ffb93f0c..e680ba0f411 100644 --- a/src/Parsers/MySQL/ASTDeclarePartitionOptions.cpp +++ b/src/Parsers/MySQL/ASTDeclarePartitionOptions.cpp @@ -44,6 +44,68 @@ ASTPtr ASTDeclarePartitionOptions::clone() const return res; } +static inline bool parsePartitionExpression(IParser::Pos & pos, std::string & type, ASTPtr & node, Expected & expected, bool subpartition = false) +{ + ASTPtr expression; + ParserExpression p_expression; + if (!subpartition && ParserKeyword("LIST").ignore(pos, expected)) + { + type = "list"; + ParserKeyword("COLUMNS").ignore(pos, expected); + if (!p_expression.parse(pos, expression, expected)) + return false; + } + else if (!subpartition && ParserKeyword("RANGE").ignore(pos, expected)) + { + type = "range"; + ParserKeyword("COLUMNS").ignore(pos, expected); + if (!p_expression.parse(pos, expression, expected)) + return false; + } + else + { + if (ParserKeyword("LINEAR").ignore(pos, expected)) + type = "linear_"; + + if (ParserKeyword("KEY").ignore(pos, expected)) + { + type += "key"; + + if (ParserKeyword("ALGORITHM").ignore(pos, expected)) + { + if (!ParserToken(TokenType::Equals).ignore(pos, expected)) + return false; + + ASTPtr algorithm; + ParserLiteral p_literal; + if (!p_literal.parse(pos, algorithm, expected) || !algorithm->as()) + return false; + + UInt64 algorithm_type = algorithm->as()->value.safeGet(); + + if (algorithm_type != 1 && algorithm_type != 2) + return false; + + type += "_" + toString(algorithm_type); + } + + if (!p_expression.parse(pos, expression, expected)) + return false; + } + else if (ParserKeyword("HASH").ignore(pos, expected)) + { + type += "hash"; + if (!p_expression.parse(pos, expression, expected)) + return false; + } + else + return false; + } + + node = expression; + return true; +} + bool ParserDeclarePartitionOptions::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { String partition_type; @@ -117,68 +179,6 @@ bool ParserDeclarePartitionOptions::parseImpl(Pos & pos, ASTPtr & node, Expected node = declare_partition_options; return true; } - -bool ParserDeclarePartitionOptions::parsePartitionExpression(Pos & pos, std::string & type, ASTPtr & node, Expected & expected, bool subpartition) -{ - ASTPtr expression; - ParserExpression p_expression; - if (!subpartition && ParserKeyword("LIST").ignore(pos, expected)) - { - type = "list"; - ParserKeyword("COLUMNS").ignore(pos, expected); - if (!p_expression.parse(pos, expression, expected)) - return false; - } - else if (!subpartition && ParserKeyword("RANGE").ignore(pos, expected)) - { - type = "range"; - ParserKeyword("COLUMNS").ignore(pos, expected); - if (!p_expression.parse(pos, expression, expected)) - return false; - } - else - { - if (ParserKeyword("LINEAR").ignore(pos, expected)) - type = "linear_"; - - if (ParserKeyword("KEY").ignore(pos, expected)) - { - type += "key"; - - if (ParserKeyword("ALGORITHM").ignore(pos, expected)) - { - if (!ParserToken(TokenType::Equals).ignore(pos, expected)) - return false; - - ASTPtr algorithm; - ParserLiteral p_literal; - if (!p_literal.parse(pos, algorithm, expected) || !algorithm->as()) - return false; - - UInt64 algorithm_type = algorithm->as()->value.safeGet(); - - if (algorithm_type != 1 && algorithm_type != 2) - return false; - - type += "_" + toString(algorithm_type); - } - - if (!p_expression.parse(pos, expression, expected)) - return false; - } - else if (ParserKeyword("HASH").ignore(pos, expected)) - { - type += "hash"; - if (!p_expression.parse(pos, expression, expected)) - return false; - } - else - return false; - } - - node = expression; - return true; -} } } diff --git a/src/Parsers/MySQL/ASTDeclarePartitionOptions.h b/src/Parsers/MySQL/ASTDeclarePartitionOptions.h index f560c66534e..3585be9df61 100644 --- a/src/Parsers/MySQL/ASTDeclarePartitionOptions.h +++ b/src/Parsers/MySQL/ASTDeclarePartitionOptions.h @@ -31,8 +31,6 @@ protected: const char * getName() const override { return "partition options declaration"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; - - bool parsePartitionExpression(Pos & pos, std::string & type, ASTPtr & node, Expected & expected, bool subpartition = false); }; } diff --git a/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml b/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml index 1acf6c64876..bfc5b6a9538 100644 --- a/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml +++ b/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml @@ -7,4 +7,4 @@ services: MYSQL_ROOT_PASSWORD: clickhouse ports: - 33307:3306 - command: --server_id=100 --log-bin='mysql-bin-1.log' + command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' diff --git a/tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml b/tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml index 5efda3252c6..7fa72c78895 100644 --- a/tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml +++ b/tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml @@ -7,4 +7,4 @@ services: MYSQL_ROOT_PASSWORD: clickhouse ports: - 33308:3306 - command: --server_id=100 --log-bin='mysql-bin-1.log' --default_authentication_plugin='mysql_native_password' + command: --server_id=100 --log-bin='mysql-bin-1.log' --default_authentication_plugin='mysql_native_password' --default-time-zone='+3:00' diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 02cbf53c072..ec98b4109ce 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -44,7 +44,7 @@ def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_nam assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", - "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" + "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n") mysql_node.query( @@ -53,28 +53,40 @@ def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_nam check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" - "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\t" + "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\t" "varchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\t0\n") mysql_node.query("UPDATE test_database.test_table_1 SET unsigned_tiny_int = 2 WHERE `key` = 1") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT key, unsigned_tiny_int, tiny_int, unsigned_small_int," + " small_int, unsigned_medium_int, medium_int, unsigned_int, _int, unsigned_integer, _integer, " + " unsigned_bigint, _bigint, unsigned_float, _float, unsigned_double, _double, _varchar, _char, " + " _date, _datetime, /* exclude it, because ON UPDATE CURRENT_TIMESTAMP _timestamp, */ " + " _bool FROM test_database.test_table_1 ORDER BY key FORMAT TSV", "1\t2\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" - "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\t" - "varchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\t0\n") + "2020-01-01 00:00:00\t1\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\t" + "varchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t0\n") # update primary key - mysql_node.query("UPDATE test_database.test_table_1 SET `key` = 3 WHERE `tiny_int` = -1") + mysql_node.query("UPDATE test_database.test_table_1 SET `key` = 3 WHERE `unsigned_tiny_int` = 2") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT key, unsigned_tiny_int, tiny_int, unsigned_small_int," + " small_int, unsigned_medium_int, medium_int, unsigned_int, _int, unsigned_integer, _integer, " + " unsigned_bigint, _bigint, unsigned_float, _float, unsigned_double, _double, _varchar, _char, " + " _date, _datetime, /* exclude it, because ON UPDATE CURRENT_TIMESTAMP _timestamp, */ " + " _bool FROM test_database.test_table_1 ORDER BY key FORMAT TSV", "2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\t" - "varchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\t0\n3\t2\t-1\t2\t-2\t3\t-3\t" - "4\t-4\t5\t-5\t6\t-6\t-1.0842022e-19\t-1.0842022e-19\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n") + "varchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t0\n3\t2\t-1\t2\t-2\t3\t-3\t" + "4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t2020-01-01 00:00:00\t1\n") mysql_node.query('DELETE FROM test_database.test_table_1 WHERE `key` = 2') - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT key, unsigned_tiny_int, tiny_int, unsigned_small_int," + " small_int, unsigned_medium_int, medium_int, unsigned_int, _int, unsigned_integer, _integer, " + " unsigned_bigint, _bigint, unsigned_float, _float, unsigned_double, _double, _varchar, _char, " + " _date, _datetime, /* exclude it, because ON UPDATE CURRENT_TIMESTAMP _timestamp, */ " + " _bool FROM test_database.test_table_1 ORDER BY key FORMAT TSV", "3\t2\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\t2020-01-01\t" - "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n") + "2020-01-01 00:00:00\t1\n") mysql_node.query('DELETE FROM test_database.test_table_1 WHERE `unsigned_tiny_int` = 2') check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", "") From 7ea2eee98af4457e2c5db973ad38186cb2a721d9 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 21 Jul 2020 17:56:55 +0800 Subject: [PATCH 114/374] ISSUES-4006 add some check & add comment --- src/Common/ErrorCodes.cpp | 2 ++ .../MySQL/DatabaseMaterializeMySQL.cpp | 12 +++++++--- .../MySQL/DatabaseMaterializeMySQL.h | 4 ++++ .../MySQL/DatabaseMaterializeTablesIterator.h | 6 +++++ src/Databases/MySQL/MaterializeMetadata.h | 8 +++++++ .../MySQL/MaterializeMySQLSettings.h | 2 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 10 ++++---- .../MySQL/MaterializeMySQLSyncThread.h | 15 +++++++++++- .../InterpreterExternalDDLQuery.cpp | 11 ++++++--- src/Parsers/ParserExternalDDLQuery.cpp | 24 +++++++++++++++++++ 10 files changed, 81 insertions(+), 13 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 0f7d9099314..bd05b346a9f 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -502,6 +502,8 @@ namespace ErrorCodes extern const int UNKNOWN_RAID_TYPE = 535; extern const int CANNOT_RESTORE_FROM_FIELD_DUMP = 536; extern const int ILLEGAL_MYSQL_VARIABLE = 537; + extern const int ILLEGAL_MYSQL_VARIABLE = 538; + extern const int MYSQL_SYNTAX_ERROR = 539; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index db657edefed..18b88f9a8f7 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -69,9 +69,8 @@ void DatabaseMaterializeMySQL::loadStoredObjects(Context & context, bool has_for { try { - LOG_DEBUG(log, "Loading MySQL nested database stored objects."); getNestedDatabase()->loadStoredObjects(context, has_force_restore_data_flag); - LOG_DEBUG(log, "Loaded MySQL nested database stored objects."); + materialize_thread.startSynchronization(); } catch (...) { @@ -220,7 +219,14 @@ bool DatabaseMaterializeMySQL::isTableExist(const String & name, const Context & StoragePtr DatabaseMaterializeMySQL::tryGetTable(const String & name, const Context & context) const { if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) - return std::make_shared(getNestedDatabase()->tryGetTable(name, context)); + { + StoragePtr nested_storage = getNestedDatabase()->tryGetTable(name, context); + + if (!nested_storage) + return {}; + + return std::make_shared(std::move(nested_storage)); + } return getNestedDatabase()->tryGetTable(name, context); } diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h index 78f8e293224..8b35a23378a 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -13,6 +13,10 @@ namespace DB { +/** Real-time pull table structure and data from remote MySQL + * + * All table structure and data will be written to the local file system + */ class DatabaseMaterializeMySQL : public IDatabase { public: diff --git a/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h b/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h index bc46e4da46e..3bad8f9c8bb 100644 --- a/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h +++ b/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h @@ -6,6 +6,12 @@ namespace DB { +/** MaterializeMySQL database table iterator + * + * The iterator returns different storage engine types depending on the visitor. + * When MySQLSync thread accesses, it always returns MergeTree + * Other cases always convert MergeTree to StorageMaterializeMySQL + */ class DatabaseMaterializeTablesIterator final : public IDatabaseTablesIterator { public: diff --git a/src/Databases/MySQL/MaterializeMetadata.h b/src/Databases/MySQL/MaterializeMetadata.h index 79769ef9f4a..c377a99a27c 100644 --- a/src/Databases/MySQL/MaterializeMetadata.h +++ b/src/Databases/MySQL/MaterializeMetadata.h @@ -8,6 +8,14 @@ namespace DB { +/** Materialize database engine metadata + * + * Record data version and current snapshot of MySQL, including: + * binlog_file - currently executing binlog_file + * binlog_position - position of the currently executing binlog file + * executed_gtid_set - currently executing gtid + * need_dumping_tables - Table structure snapshot at the current moment(Only when database first created or executed binlog file is deleted) + */ struct MaterializeMetadata { const String persistent_path; diff --git a/src/Databases/MySQL/MaterializeMySQLSettings.h b/src/Databases/MySQL/MaterializeMySQLSettings.h index 6f64865afac..de6a1ae1ede 100644 --- a/src/Databases/MySQL/MaterializeMySQLSettings.h +++ b/src/Databases/MySQL/MaterializeMySQLSettings.h @@ -8,7 +8,7 @@ namespace DB class ASTStorage; -/** Settings for the MySQL Database engine(materialize mode). +/** Settings for the MaterializeMySQL database engine. * Could be loaded from a CREATE DATABASE query (SETTINGS clause). */ struct MaterializeMySQLSettings : public SettingsCollection diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index fc105ce177d..4ec6bc320b5 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -144,9 +144,7 @@ MaterializeMySQLSyncThread::MaterializeMySQLSyncThread( : log(&Poco::Logger::get("MaterializeMySQLSyncThread")), global_context(context.getGlobalContext()), database_name(database_name_) , mysql_database_name(mysql_database_name_), pool(std::move(pool_)), client(std::move(client_)), settings(settings_) { - const auto & mysql_server_version = checkVariableAndGetVersion(pool.get()); query_prefix = "EXTERNAL DDL FROM MySQL(" + backQuoteIfNeed(database_name) + ", " + backQuoteIfNeed(mysql_database_name) + ") "; - startSynchronization(mysql_server_version); } void MaterializeMySQLSyncThread::synchronization(const String & mysql_version) @@ -210,10 +208,12 @@ void MaterializeMySQLSyncThread::stopSynchronization() } } -void MaterializeMySQLSyncThread::startSynchronization(const String & mysql_version) +void MaterializeMySQLSyncThread::startSynchronization() { - /// TODO: reset exception. - background_thread_pool = std::make_unique([this, mysql_version = mysql_version]() { synchronization(mysql_version); }); + const auto & mysql_server_version = checkVariableAndGetVersion(pool.get()); + + background_thread_pool = std::make_unique( + [this, mysql_server_version = mysql_server_version]() { synchronization(mysql_server_version); }); } static inline void cleanOutdatedTables(const String & database_name, const Context & context) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index 4d4dbc28624..f244c24ab9a 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -21,6 +21,19 @@ namespace DB { +/** MySQL table structure and data synchronization thread + * + * When catch exception, it always exits immediately. + * In this case, you need to execute DETACH DATABASE and ATTACH DATABASE after manual processing + * + * The whole work of the thread includes synchronous full data and real-time pull incremental data + * + * synchronous full data: + * We will synchronize the full data when the database is first create or not found binlog file in MySQL after restart. + * + * real-time pull incremental data: + * We will pull the binlog event of MySQL to parse and execute when the full data synchronization is completed. + */ class MaterializeMySQLSyncThread { public: @@ -32,7 +45,7 @@ public: void stopSynchronization(); - void startSynchronization(const String & mysql_version); + void startSynchronization(); static bool isMySQLSyncThread(); diff --git a/src/Interpreters/InterpreterExternalDDLQuery.cpp b/src/Interpreters/InterpreterExternalDDLQuery.cpp index 2e9c594812b..8f1c9da22b7 100644 --- a/src/Interpreters/InterpreterExternalDDLQuery.cpp +++ b/src/Interpreters/InterpreterExternalDDLQuery.cpp @@ -11,9 +11,10 @@ #include #ifdef USE_MYSQL +# include +# include # include # include -# include #endif namespace DB @@ -21,6 +22,7 @@ namespace DB namespace ErrorCodes { + extern const int SYNTAX_ERROR; extern const int BAD_ARGUMENTS; } @@ -33,6 +35,9 @@ BlockIO InterpreterExternalDDLQuery::execute() { const ASTExternalDDLQuery & external_ddl_query = query->as(); + if (context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY) + throw Exception("Cannot parse and execute EXTERNAL DDL FROM.", ErrorCodes::SYNTAX_ERROR); + if (external_ddl_query.from->name == "MySQL") { #ifdef USE_MYSQL @@ -53,11 +58,11 @@ BlockIO InterpreterExternalDDLQuery::execute() else if (external_ddl_query.external_ddl->as()) return MySQLInterpreter::InterpreterMySQLAlterQuery( external_ddl_query.external_ddl, context, getIdentifierName(arguments[0]), - getIdentifierName(arguments[1])) .execute(); + getIdentifierName(arguments[1])).execute(); else if (external_ddl_query.external_ddl->as()) return MySQLInterpreter::InterpreterMySQLCreateQuery( external_ddl_query.external_ddl, context, getIdentifierName(arguments[0]), - getIdentifierName(arguments[1])) .execute(); + getIdentifierName(arguments[1])).execute(); } return BlockIO(); diff --git a/src/Parsers/ParserExternalDDLQuery.cpp b/src/Parsers/ParserExternalDDLQuery.cpp index 20020da22bd..b6ada39a800 100644 --- a/src/Parsers/ParserExternalDDLQuery.cpp +++ b/src/Parsers/ParserExternalDDLQuery.cpp @@ -17,6 +17,13 @@ namespace DB { +#ifdef USE_MYSQL +namespace ErrorCodes +{ + extern const int MYSQL_SYNTAX_ERROR; +} +#endif + bool ParserExternalDDLQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ParserFunction p_function; @@ -49,6 +56,23 @@ bool ParserExternalDDLQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect if (external_ddl_query->external_ddl) external_ddl_query->children.push_back(external_ddl_query->external_ddl); + + if (!res) + { + /// Syntax error is ignored, so we need to convert the error code for parsing failure + + if (ParserKeyword("ALTER TABLE").ignore(pos)) + throw Exception("Cannot parse MySQL alter query.", ErrorCodes::MYSQL_SYNTAX_ERROR); + + if (ParserKeyword("RENAME TABLE").ignore(pos)) + throw Exception("Cannot parse MySQL rename query.", ErrorCodes::MYSQL_SYNTAX_ERROR); + + if (ParserKeyword("DROP TABLE").ignore(pos) || ParserKeyword("TRUNCATE").ignore(pos)) + throw Exception("Cannot parse MySQL drop query.", ErrorCodes::MYSQL_SYNTAX_ERROR); + + if (ParserKeyword("CREATE TABLE").ignore(pos) || ParserKeyword("CREATE TEMPORARY TABLE").ignore(pos)) + throw Exception("Cannot parse MySQL create query.", ErrorCodes::MYSQL_SYNTAX_ERROR); + } #endif } From 1eddb3fbc8c8f74254a19c8e0b49bfd555fd615f Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 23 Jul 2020 16:17:07 +0800 Subject: [PATCH 115/374] ISSUES-4006 fix build failure after merge master --- src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 2 +- src/Storages/StorageMaterializeMySQL.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 7e5008569de..682327699e3 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -92,7 +92,7 @@ static NamesAndTypesList getNames(const ASTFunction & expr, const Context & cont return NamesAndTypesList{}; ASTPtr temp_ast = expr.clone(); - auto syntax = SyntaxAnalyzer(context).analyze(temp_ast, columns); + auto syntax = TreeRewriter(context).analyze(temp_ast, columns); auto expression = ExpressionAnalyzer(temp_ast, syntax, context).getActions(false); return expression->getRequiredColumnsWithTypes(); } diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index 5d4ec84ecf5..880571d4ddc 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -80,7 +80,7 @@ Pipes StorageMaterializeMySQL::read( if (!expressions->children.empty() && !pipes.empty()) { Block pipe_header = pipes.front().getHeader(); - SyntaxAnalyzerResultPtr syntax = SyntaxAnalyzer(context).analyze(expressions, pipe_header.getNamesAndTypesList()); + auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList()); ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true); for (auto & pipe : pipes) From da2ad0799f575cb4442c575f147cf8610e953490 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 3 Aug 2020 17:11:46 +0800 Subject: [PATCH 116/374] ISSUES-4006 try fix build failure after merge master --- src/Common/ErrorCodes.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index bd05b346a9f..b75f885e559 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -502,8 +502,7 @@ namespace ErrorCodes extern const int UNKNOWN_RAID_TYPE = 535; extern const int CANNOT_RESTORE_FROM_FIELD_DUMP = 536; extern const int ILLEGAL_MYSQL_VARIABLE = 537; - extern const int ILLEGAL_MYSQL_VARIABLE = 538; - extern const int MYSQL_SYNTAX_ERROR = 539; + extern const int MYSQL_SYNTAX_ERROR = 538; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; From cf78a1f8fc06040743b554d48c3fe2914032732e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 3 Aug 2020 17:07:47 +0300 Subject: [PATCH 117/374] try fix build --- .../MySQL/MaterializeMySQLSettings.cpp | 3 +-- .../MySQL/MaterializeMySQLSettings.h | 25 ++++++++++--------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMySQLSettings.cpp b/src/Databases/MySQL/MaterializeMySQLSettings.cpp index fba4eaae2fa..688de941692 100644 --- a/src/Databases/MySQL/MaterializeMySQLSettings.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSettings.cpp @@ -2,7 +2,6 @@ #include #include -#include namespace DB { @@ -13,7 +12,7 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -IMPLEMENT_SETTINGS_COLLECTION(MaterializeMySQLSettings, LIST_OF_MATERIALIZE_MODE_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(MaterializeMySQLSettingsTraits, LIST_OF_MATERIALIZE_MODE_SETTINGS) void MaterializeMySQLSettings::loadFromQuery(ASTStorage & storage_def) { diff --git a/src/Databases/MySQL/MaterializeMySQLSettings.h b/src/Databases/MySQL/MaterializeMySQLSettings.h index de6a1ae1ede..effd231ed45 100644 --- a/src/Databases/MySQL/MaterializeMySQLSettings.h +++ b/src/Databases/MySQL/MaterializeMySQLSettings.h @@ -1,28 +1,29 @@ #pragma once #include -#include +#include namespace DB { class ASTStorage; +#define LIST_OF_MATERIALIZE_MODE_SETTINGS(M) \ + M(UInt64, max_rows_in_buffer, DEFAULT_BLOCK_SIZE, "", 0) \ + M(UInt64, max_bytes_in_buffer, DBMS_DEFAULT_BUFFER_SIZE, "", 0) \ + M(UInt64, max_rows_in_buffers, DEFAULT_BLOCK_SIZE, "", 0) \ + M(UInt64, max_bytes_in_buffers, DBMS_DEFAULT_BUFFER_SIZE, "", 0) \ + M(UInt64, max_flush_data_time, 1000, "", 0) \ + M(UInt64, max_wait_time_when_mysql_unavailable, 1000, "", 0) \ + + DECLARE_SETTINGS_TRAITS(MaterializeMySQLSettingsTraits, LIST_OF_MATERIALIZE_MODE_SETTINGS) + + /** Settings for the MaterializeMySQL database engine. * Could be loaded from a CREATE DATABASE query (SETTINGS clause). */ -struct MaterializeMySQLSettings : public SettingsCollection +struct MaterializeMySQLSettings : public BaseSettings { -#define LIST_OF_MATERIALIZE_MODE_SETTINGS(M) \ - M(SettingUInt64, max_rows_in_buffer, DEFAULT_BLOCK_SIZE, "", 0) \ - M(SettingUInt64, max_bytes_in_buffer, DBMS_DEFAULT_BUFFER_SIZE, "", 0) \ - M(SettingUInt64, max_rows_in_buffers, DEFAULT_BLOCK_SIZE, "", 0) \ - M(SettingUInt64, max_bytes_in_buffers, DBMS_DEFAULT_BUFFER_SIZE, "", 0) \ - M(SettingUInt64, max_flush_data_time, 1000, "", 0) \ - M(SettingUInt64, max_wait_time_when_mysql_unavailable, 1000, "", 0) \ - - DECLARE_SETTINGS_COLLECTION(LIST_OF_MATERIALIZE_MODE_SETTINGS) - void loadFromQuery(ASTStorage & storage_def); }; From 34f7fac76c8a1829e4d138dd019a2f8d945ab804 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 4 Aug 2020 12:22:12 +0800 Subject: [PATCH 118/374] ISSUES-4006 try fix test failure --- src/Core/MySQLClient.cpp | 4 ++-- src/Core/MySQLReplication.cpp | 1 - src/Core/tests/mysql_protocol.cpp | 20 +++++++++---------- src/Databases/DatabaseFactory.cpp | 2 +- .../MySQL/DatabaseMaterializeMySQL.cpp | 7 +++++-- .../MySQL/DatabaseMaterializeMySQL.h | 3 +-- .../MySQL/tests/gtest_create_rewritten.cpp | 15 +++++++------- src/Parsers/MySQL/ASTCreateQuery.cpp | 7 ++++--- 8 files changed, 31 insertions(+), 28 deletions(-) diff --git a/src/Core/MySQLClient.cpp b/src/Core/MySQLClient.cpp index 06c26f200e4..95e83971f2b 100644 --- a/src/Core/MySQLClient.cpp +++ b/src/Core/MySQLClient.cpp @@ -129,8 +129,8 @@ void MySQLClient::startBinlogDump(UInt32 slave_id, String replicate_db, String b String checksum = "CRC32"; writeCommand(Command::COM_QUERY, "SET @master_binlog_checksum = '" + checksum + "'"); - /// Set heartbeat 30s. - UInt64 period_ns = (30 * 1e9); + /// Set heartbeat 1s. + UInt64 period_ns = (1 * 1e9); writeCommand(Command::COM_QUERY, "SET @master_heartbeat_period = " + std::to_string(period_ns)); // Register slave. diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 1933a11b036..934a262b36c 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -774,7 +774,6 @@ namespace MySQLReplication event = std::make_shared(); event->parseHeader(payload); event->parseEvent(payload); - position.updateLogPos(event->header.log_pos); table_map = std::static_pointer_cast(event); break; } diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index a57773e4eb9..c86cf0be25a 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -16,7 +16,7 @@ int main(int argc, char ** argv) String user = "default"; String password = "123"; - String database = ""; + String database; UInt8 charset_utf8 = 33; UInt32 max_packet_size = MySQLProtocol::MAX_PACKET_LENGTH; @@ -113,7 +113,7 @@ int main(int argc, char ** argv) ASSERT(client.err.header == server.header) ASSERT(client.err.error_code == server.error_code) ASSERT(client.err.sql_state == server.sql_state) - ASSERT(client.err.error_message.data() == server.error_message) + ASSERT(client.err.error_message == server.error_message) } /// EOF Packet @@ -204,32 +204,32 @@ int main(int argc, char ** argv) switch (event->type()) { case MYSQL_QUERY_EVENT: { - auto binlogEvent = std::static_pointer_cast(event); - binlogEvent->dump(); + auto binlog_event = std::static_pointer_cast(event); + binlog_event->dump(); Position pos = slave.getPosition(); std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; break; } case MYSQL_WRITE_ROWS_EVENT: { - auto binlogEvent = std::static_pointer_cast(event); - binlogEvent->dump(); + auto binlog_event = std::static_pointer_cast(event); + binlog_event->dump(); Position pos = slave.getPosition(); std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; break; } case MYSQL_UPDATE_ROWS_EVENT: { - auto binlogEvent = std::static_pointer_cast(event); - binlogEvent->dump(); + auto binlog_event = std::static_pointer_cast(event); + binlog_event->dump(); Position pos = slave.getPosition(); std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; break; } case MYSQL_DELETE_ROWS_EVENT: { - auto binlogEvent = std::static_pointer_cast(event); - binlogEvent->dump(); + auto binlog_event = std::static_pointer_cast(event); + binlog_event->dump(); Position pos = slave.getPosition(); std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index cbd5f055f31..4553e648804 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -74,7 +74,7 @@ static inline ValueType safeGetLiteralValue(const ASTPtr &ast, const String &eng DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String & metadata_path, Context & context) { - const auto * engine_define = create.storage; + auto * engine_define = create.storage; const String & database_name = create.database; const String & engine_name = engine_define->engine->name; const UUID & uuid = create.uuid; diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 18b88f9a8f7..031384bef2c 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -162,7 +162,7 @@ StoragePtr DatabaseMaterializeMySQL::detachTable(const String & name) return getNestedDatabase()->detachTable(name); } -void DatabaseMaterializeMySQL::renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange) +void DatabaseMaterializeMySQL::renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) { if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) throw Exception("MaterializeMySQL database not support rename table.", ErrorCodes::NOT_IMPLEMENTED); @@ -170,10 +170,13 @@ void DatabaseMaterializeMySQL::renameTable(const Context & context, const String if (exchange) throw Exception("MaterializeMySQL database not support exchange table.", ErrorCodes::NOT_IMPLEMENTED); + if (dictionary) + throw Exception("MaterializeMySQL database not support rename dictionary.", ErrorCodes::NOT_IMPLEMENTED); + if (to_database.getDatabaseName() != getDatabaseName()) throw Exception("Cannot rename with other database for MaterializeMySQL database.", ErrorCodes::NOT_IMPLEMENTED); - getNestedDatabase()->renameTable(context, name, *getNestedDatabase(), to_name, exchange); + getNestedDatabase()->renameTable(context, name, *getNestedDatabase(), to_name, exchange, dictionary); } void DatabaseMaterializeMySQL::alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h index 8b35a23378a..c1fd0bb77a0 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -36,7 +36,6 @@ protected: Poco::Logger * log; MaterializeMySQLSyncThread materialize_thread; - mutable std::mutex mutex; std::exception_ptr exception; DatabasePtr getNestedDatabase() const; @@ -68,7 +67,7 @@ public: StoragePtr detachTable(const String & name) override; - void renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange) override; + void renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) override; void alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index 1a1568c0863..6e037593e7e 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -12,14 +12,16 @@ #include #include #include +#include using namespace DB; -static inline ASTPtr tryRewrittenCreateQuery(const String & query, Context & context) +static inline ASTPtr tryRewrittenCreateQuery(const String & query, const Context & context_) { ParserExternalDDLQuery external_ddl_parser; ASTPtr ast = parseQuery(external_ddl_parser, query, 0, 0); + Context context = context_; context.unsafeSetCurrentDatabase("default"); return MySQLInterpreter::InterpreterCreateImpl::getRewrittenQuery( *ast->as()->external_ddl->as(), @@ -29,30 +31,29 @@ static inline ASTPtr tryRewrittenCreateQuery(const String & query, Context & con TEST(MySQLCreateRewritten, RewrittenQueryWithPrimaryKey) { registerFunctions(); - auto shared_context = Context::createShared(); - auto global_context = std::make_unique(Context::createGlobal(shared_context.get())); + const auto & context_holder = getContext(); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL PRIMARY " - "KEY) ENGINE=InnoDB DEFAULT CHARSET=utf8", *global_context)), + "KEY) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), "CREATE TABLE test_database.test_table_1 (`key` Int32, `_sign` Int8, `_version` UInt64) ENGINE = ReplacingMergeTree(_version) " "PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL, " - " PRIMARY KEY (`key`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", *global_context)), + " PRIMARY KEY (`key`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), "CREATE TABLE test_database.test_table_1 (`key` Int32, `_sign` Int8, `_version` UInt64) ENGINE = ReplacingMergeTree(_version) " "PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key_1` int NOT NULL, " - " key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", *global_context)), + " key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), "CREATE TABLE test_database.test_table_1 (`key_1` Int32, `key_2` Int32, `_sign` Int8, `_version` UInt64) ENGINE = " "ReplacingMergeTree(_version) PARTITION BY intDiv(key_1, 4294967) ORDER BY (key_1, key_2)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key_1` BIGINT NOT NULL, " - " key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", *global_context)), + " key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), "CREATE TABLE test_database.test_table_1 (`key_1` Int64, `key_2` Int32, `_sign` Int8, `_version` UInt64) ENGINE = " "ReplacingMergeTree(_version) PARTITION BY intDiv(key_2, 4294967) ORDER BY (key_1, key_2)"); } diff --git a/src/Parsers/MySQL/ASTCreateQuery.cpp b/src/Parsers/MySQL/ASTCreateQuery.cpp index a200ce2e0ff..ad527b91ec5 100644 --- a/src/Parsers/MySQL/ASTCreateQuery.cpp +++ b/src/Parsers/MySQL/ASTCreateQuery.cpp @@ -63,7 +63,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (ParserKeyword("IF NOT EXISTS").ignore(pos, expected)) if_not_exists = true; - if (!ParserCompoundIdentifier(true).parse(pos, table, expected)) + if (!ParserCompoundIdentifier(false).parse(pos, table, expected)) return false; if (ParserKeyword("LIKE").ignore(pos, expected)) @@ -71,8 +71,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!ParserCompoundIdentifier(true).parse(pos, like_table, expected)) return false; } - - if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)) + else if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)) { if (ParserKeyword("LIKE").ignore(pos, expected)) { @@ -94,6 +93,8 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserDeclarePartitionOptions().parse(pos, partition_options, expected); } } + else + return false; auto create_query = std::make_shared(); From 0e3108cdf0c47220463ef21c739b27be4617d8c8 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 4 Aug 2020 12:26:16 +0800 Subject: [PATCH 119/374] ISSUES-4006 fix parser use uuid --- src/Parsers/MySQL/ASTCreateQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/MySQL/ASTCreateQuery.cpp b/src/Parsers/MySQL/ASTCreateQuery.cpp index ad527b91ec5..6c44a915f73 100644 --- a/src/Parsers/MySQL/ASTCreateQuery.cpp +++ b/src/Parsers/MySQL/ASTCreateQuery.cpp @@ -68,14 +68,14 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (ParserKeyword("LIKE").ignore(pos, expected)) { - if (!ParserCompoundIdentifier(true).parse(pos, like_table, expected)) + if (!ParserCompoundIdentifier(false).parse(pos, like_table, expected)) return false; } else if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)) { if (ParserKeyword("LIKE").ignore(pos, expected)) { - if (!ParserCompoundIdentifier(true).parse(pos, like_table, expected)) + if (!ParserCompoundIdentifier(false).parse(pos, like_table, expected)) return false; if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) From c99da34039d9099294ab9b6f7a39fd912206adaa Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 4 Aug 2020 19:06:11 +0800 Subject: [PATCH 120/374] ISSUES-4006 try fix unit test failure --- .../tests/gtest_global_register_functions.h | 17 +++++++++++++++++ .../MySQL/tests/gtest_create_rewritten.cpp | 4 ++-- ...st_transform_query_for_external_database.cpp | 4 ++-- 3 files changed, 21 insertions(+), 4 deletions(-) create mode 100644 src/Common/tests/gtest_global_register_functions.h diff --git a/src/Common/tests/gtest_global_register_functions.h b/src/Common/tests/gtest_global_register_functions.h new file mode 100644 index 00000000000..197ce5838b9 --- /dev/null +++ b/src/Common/tests/gtest_global_register_functions.h @@ -0,0 +1,17 @@ +#include +#include + +struct RegisteredFunctionsState +{ + RegisteredFunctionsState() + { + DB::registerFunctions(); + } + + RegisteredFunctionsState(RegisteredFunctionsState &&) = default; +}; + +inline void tryRegisterFunctions() +{ + static RegisteredFunctionsState registered_functions_state; +} diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index 6e037593e7e..62683dc546d 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -10,9 +10,9 @@ #include #include #include -#include #include #include +#include using namespace DB; @@ -30,7 +30,7 @@ static inline ASTPtr tryRewrittenCreateQuery(const String & query, const Context TEST(MySQLCreateRewritten, RewrittenQueryWithPrimaryKey) { - registerFunctions(); + tryRegisterFunctions(); const auto & context_holder = getContext(); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 1a63132ee0c..31fc49582ad 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -9,8 +9,8 @@ #include #include #include -#include #include +#include using namespace DB; @@ -42,7 +42,7 @@ private: explicit State() : context(getContext().context) { - registerFunctions(); + tryRegisterFunctions(); DatabasePtr database = std::make_shared("test", context); database->attachTable("table", StorageMemory::create(StorageID("test", "table"), ColumnsDescription{columns}, ConstraintsDescription{})); DatabaseCatalog::instance().attachDatabase("test", database); From 9385f3de0ea4cba50817dfafe78f80cf19e6b60e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Aug 2020 16:06:59 +0300 Subject: [PATCH 121/374] Refactor Pipe [part 4]. --- .../tests/finish_sorting_stream.cpp | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 6 +- src/Interpreters/MergeJoin.cpp | 2 +- src/Processors/IProcessor.h | 5 - src/Processors/Pipe.cpp | 221 ++++++----- src/Processors/Pipe.h | 125 +----- src/Processors/QueryPipeline.cpp | 375 +++--------------- src/Processors/QueryPipeline.h | 110 ++--- src/Processors/QueryPlan/AggregatingStep.cpp | 6 +- .../QueryPlan/FinishSortingStep.cpp | 4 +- src/Processors/QueryPlan/LimitStep.cpp | 2 +- .../QueryPlan/MergingAggregatedStep.cpp | 10 +- .../QueryPlan/MergingSortedStep.cpp | 4 +- src/Processors/QueryPlan/OffsetStep.cpp | 2 +- src/Processors/Sources/DelayedSource.cpp | 30 +- .../Transforms/AggregatingTransform.cpp | 26 +- ...gingAggregatedMemoryEfficientTransform.cpp | 46 +-- ...ergingAggregatedMemoryEfficientTransform.h | 10 +- 18 files changed, 269 insertions(+), 717 deletions(-) diff --git a/src/DataStreams/tests/finish_sorting_stream.cpp b/src/DataStreams/tests/finish_sorting_stream.cpp index ea151e01293..7ce342d3e8e 100644 --- a/src/DataStreams/tests/finish_sorting_stream.cpp +++ b/src/DataStreams/tests/finish_sorting_stream.cpp @@ -72,7 +72,7 @@ int main(int argc, char ** argv) QueryPipeline pipeline; pipeline.init(std::move(source)); - pipeline.addPipe({std::make_shared(pipeline.getHeader(), sort_descr, n, 0, 0, 0, nullptr, 0)}); + pipeline.addTransform(std::make_shared(pipeline.getHeader(), sort_descr, n, 0, 0, 0, nullptr, 0)); SortDescription sort_descr_final; sort_descr_final.emplace_back("col1", 1, 1); diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 1a94225443c..4fed6f02d5f 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -269,10 +269,12 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() if (settings.graph) { + auto processors = Pipe::detachProcessors(std::move(*pipeline).getPipe()); + if (settings.compact) - printPipelineCompact(pipeline->getProcessors(), buffer, settings.query_pipeline_options.header); + printPipelineCompact(processors, buffer, settings.query_pipeline_options.header); else - printPipeline(pipeline->getProcessors(), buffer); + printPipeline(processors, buffer); } else { diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 63e7aa36535..e279b741c91 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -516,7 +516,7 @@ void MergeJoin::mergeInMemoryRightBlocks() pipeline.init(std::move(source)); /// TODO: there should be no splitted keys by blocks for RIGHT|FULL JOIN - pipeline.addPipe({std::make_shared(pipeline.getHeader(), right_sort_description, max_rows_in_right_block, 0, 0, 0, nullptr, 0)}); + pipeline.addTransform(std::make_shared(pipeline.getHeader(), right_sort_description, max_rows_in_right_block, 0, 0, 0, nullptr, 0)); auto sorted_input = PipelineExecutingBlockInputStream(std::move(pipeline)); diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index e9148dd5075..40c3c8c8940 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -287,9 +287,6 @@ public: size_t getStream() const { return stream_number; } constexpr static size_t NO_STREAM = std::numeric_limits::max(); - void enableQuota() { has_quota = true; } - bool hasQuota() const { return has_quota; } - /// Step of QueryPlan from which processor was created. void setQueryPlanStep(IQueryPlanStep * step, size_t group = 0) { @@ -310,8 +307,6 @@ private: size_t stream_number = NO_STREAM; - bool has_quota = false; - IQueryPlanStep * query_plan_step = nullptr; size_t query_plan_step_group = 0; }; diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index b6c3ecb5f88..65c2e5f2a5a 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -123,6 +123,59 @@ static OutputPort * uniteTotals(const OutputPortRawPtrs & ports, const Block & h return totals_port; } +Pipe::Pipe(ProcessorPtr source, OutputPort * output, OutputPort * totals, OutputPort * extremes) +{ + if (!source->getInputs().empty()) + throw Exception("Source for pipe shouldn't have any input, but " + source->getName() + " has " + + toString(source->getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR); + + if (!output) + throw Exception("Cannot create Pipe from source because specified output port is nullptr", + ErrorCodes::LOGICAL_ERROR); + + if (output == totals || output == extremes || (totals && totals == extremes)) + throw Exception("Cannot create Pipe from source because some of specified ports are the same", + ErrorCodes::LOGICAL_ERROR); + + header = output->getHeader(); + + /// Check that ports belong to source and all ports from source were specified. + { + auto & outputs = source->getOutputs(); + size_t num_specified_ports = 0; + + auto check_port_from_source = [&](OutputPort * port, std::string name) + { + if (!port) + return; + + assertBlocksHaveEqualStructure(header, port->getHeader(), name); + + ++num_specified_ports; + + auto it = std::find_if(outputs.begin(), outputs.end(), [port](const OutputPort & p) { return &p == port; }); + if (it == outputs.end()) + throw Exception("Cannot create Pipe because specified " + name + " port does not belong to source", + ErrorCodes::LOGICAL_ERROR); + }; + + check_port_from_source(output, "output"); + check_port_from_source(totals, "totals"); + check_port_from_source(extremes, "extremes"); + + if (num_specified_ports != outputs.size()) + throw Exception("Cannot create Pipe from source because it has " + std::to_string(outputs.size()) + + " output ports, but " + std::to_string(num_specified_ports) + " were specified", + ErrorCodes::LOGICAL_ERROR); + } + + totals_port = totals; + extremes_port = extremes; + output_ports.push_back(output); + processors.emplace_back(std::move(source)); + max_parallel_streams = 1; +} + Pipe::Pipe(ProcessorPtr source) { if (auto * source_from_input_stream = typeid_cast(source.get())) @@ -262,51 +315,24 @@ Pipe Pipe::unitePipes(Pipes pipes, Processors * collected_processors) } } -//void Pipe::addPipes(Pipe pipes) -//{ -// if (processors.empty()) -// { -// *this = std::move(pipes); -// return; -// } -// -// if (pipes.processors.empty()) -// return; -// -// assertBlocksHaveEqualStructure(header, pipes.header, "Pipe"); -// -// max_parallel_streams += pipes.max_parallel_streams; -// processors.insert(processors.end(), pipes.processors.begin(), pipes.processors.end()); -// -// OutputPortRawPtrs totals; -// if (totals_port) -// totals.emplace_back(totals_port); -// if (pipes.totals_port) -// totals.emplace_back(pipes.totals_port); -// if (!totals.empty()) -// totals_port = uniteTotals(totals, header, processors); -// -// OutputPortRawPtrs extremes; -// if (extremes_port) -// extremes.emplace_back(extremes_port); -// if (pipes.extremes_port) -// extremes.emplace_back(pipes.extremes_port); -// if (!extremes.empty()) -// extremes_port = uniteExtremes(extremes, header, processors); -//} +void Pipe::addSource(ProcessorPtr source) +{ + checkSource(*source); + const auto & source_header = output_ports.front()->getHeader(); -//void Pipe::addSource(ProcessorPtr source) -//{ -// checkSource(*source); -// const auto & source_header = output_ports.front()->getHeader(); -// -// assertBlocksHaveEqualStructure(header, source_header, "Pipes"); !!!! -// -// output_ports.push_back(&source->getOutputs().front()); -// processors.emplace_back(std::move(source)); -// -// max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); -//} + if (output_ports.empty()) + header = source_header; + else + assertBlocksHaveEqualStructure(header, source_header, "Pipes"); + + if (collected_processors) + collected_processors->emplace_back(source.get()); + + output_ports.push_back(&source->getOutputs().front()); + processors.emplace_back(std::move(source)); + + max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); +} void Pipe::addTotalsSource(ProcessorPtr source) { @@ -456,6 +482,48 @@ void Pipe::addSimpleTransform(const ProcessorGetter & getter) addSimpleTransform([&](const Block & stream_header, StreamType) { return getter(stream_header); }); } +void Pipe::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) +{ + if (output_ports.empty()) + throw Exception("Cannot set sink to empty Pipe.", ErrorCodes::LOGICAL_ERROR); + + auto add_transform = [&](OutputPort *& stream, Pipe::StreamType stream_type) + { + if (!stream) + return; + + auto transform = getter(stream->getHeader(), stream_type); + + if (transform) + { + if (transform->getInputs().size() != 1) + throw Exception("Sink for query pipeline transform should have single input, " + "but " + transform->getName() + " has " + + toString(transform->getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR); + + if (!transform->getOutputs().empty()) + throw Exception("Sink for query pipeline transform should have no outputs, " + "but " + transform->getName() + " has " + + toString(transform->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); + } + + if (!transform) + transform = std::make_shared(stream->getHeader()); + + connect(*stream, transform->getInputs().front()); + processors.emplace_back(std::move(transform)); + }; + + for (auto & port : output_ports) + add_transform(port, StreamType::Main); + + add_transform(totals_port, StreamType::Totals); + add_transform(extremes_port, StreamType::Extremes); + + output_ports.clear(); + header.clear(); +} + void Pipe::transform(const Transformer & transformer) { if (output_ports.empty()) @@ -561,69 +629,4 @@ void Pipe::enableQuota() } } -/* -Pipe::Pipe(ProcessorPtr source) -{ - if (auto * source_from_input_stream = typeid_cast(source.get())) - { - totals = source_from_input_stream->getTotalsPort(); - extremes = source_from_input_stream->getExtremesPort(); - } - else if (source->getOutputs().size() != 1) - checkSource(*source); - - output_port = &source->getOutputs().front(); - - processors.emplace_back(std::move(source)); - max_parallel_streams = 1; -} - -Pipe::Pipe(Processors processors_, OutputPort * output_port_, OutputPort * totals_, OutputPort * extremes_) - : processors(std::move(processors_)), output_port(output_port_), totals(totals_), extremes(extremes_) -{ -} - -Pipe::Pipe(Pipes && pipes, ProcessorPtr transform) -{ - checkSingleOutput(*transform); - checkMultipleInputs(*transform, pipes.size()); - - auto it = transform->getInputs().begin(); - - for (auto & pipe : pipes) - { - connect(*pipe.output_port, *it); - ++it; - - max_parallel_streams += pipe.max_parallel_streams; - processors.insert(processors.end(), pipe.processors.begin(), pipe.processors.end()); - - std::move(pipe.table_locks.begin(), pipe.table_locks.end(), std::back_inserter(table_locks)); - std::move(pipe.interpreter_context.begin(), pipe.interpreter_context.end(), std::back_inserter(interpreter_context)); - std::move(pipe.storage_holders.begin(), pipe.storage_holders.end(), std::back_inserter(storage_holders)); - } - - output_port = &transform->getOutputs().front(); - processors.emplace_back(std::move(transform)); -} - -Pipe::Pipe(OutputPort * port) : output_port(port) -{ -} - -void Pipe::addProcessors(const Processors & processors_) -{ - processors.insert(processors.end(), processors_.begin(), processors_.end()); -} - -void Pipe::addSimpleTransform(ProcessorPtr transform) -{ - checkSimpleTransform(*transform); - connect(*output_port, transform->getInputs().front()); - output_port = &transform->getOutputs().front(); - processors.emplace_back(std::move(transform)); -} - - -*/ } diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 5854f61554a..e6ae375da48 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -25,6 +25,8 @@ public: Pipe() = default; /// Create from source. Source must have no input ports and single output. explicit Pipe(ProcessorPtr source); + /// Create from source with specified totals end extremes (may be nullptr). Ports should be owned by source. + explicit Pipe(ProcessorPtr source, OutputPort * output, OutputPort * totals, OutputPort * extremes); /// Create from processors. Use all not-connected output ports as output_ports. Check invariants. explicit Pipe(Processors processors_); @@ -34,19 +36,17 @@ public: Pipe & operator=(Pipe && other) = default; const Block & getHeader() const { return header; } - bool empty() const { return output_ports.empty(); } + bool empty() const { return processors.empty(); } size_t numOutputPorts() const { return output_ports.size(); } + size_t maxParallelStreams() const { return max_parallel_streams; } OutputPort * getOutputPort(size_t pos) const { return output_ports[pos]; } OutputPort * getTotalsPort() const { return totals_port; } OutputPort * getExtremesPort() const { return extremes_port; } - /// Add processors form other pipe. It should have same header. - //void addPipes(Pipes pipes); - /// Add processor to list, add it output ports to output_ports. /// Processor shouldn't have input ports, output ports shouldn't be connected. /// Output headers should have same structure and be compatible with current header (if not empty()). - /// void addSource(ProcessorPtr source); + void addSource(ProcessorPtr source); /// Add totals and extremes. void addTotalsSource(ProcessorPtr source); @@ -79,6 +79,9 @@ public: /// Unite several pipes together. They should have same header. static Pipe unitePipes(Pipes pipes); + /// Get processors from Pipe. Use it with cautious, it is easy to loss totals and extremes ports. + static Processors detachProcessors(Pipe pipe) { return std::move(pipe.processors); } + /// Specify quotas and limits for every ISourceWithProgress. void setLimits(const SourceWithProgress::LocalLimits & limits); void setQuota(const std::shared_ptr & quota); @@ -91,123 +94,39 @@ public: void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); } private: - Processors processors; + /// Destruction order: processors, header, locks, temporary storages, local contexts + + /// Some processors may implicitly use Context or temporary Storage created by Interpreter. + /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, + /// because QueryPipeline is alive until query is finished. + std::vector> interpreter_context; + std::vector storage_holders; + std::vector table_locks; /// Header is common for all output below. Block header; + Processors processors; /// Output ports. Totals and extremes are allowed to be empty. OutputPortRawPtrs output_ports; OutputPort * totals_port = nullptr; OutputPort * extremes_port = nullptr; - /// It is the max number of processors which can be executed in parallel for each step. See QueryPipeline::Streams. + /// It is the max number of processors which can be executed in parallel for each step. /// Usually, it's the same as the number of output ports. size_t max_parallel_streams = 0; - std::vector table_locks; - - /// Some processors may implicitly use Context or temporary Storage created by Interpreter. - /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, - /// because QueryPipeline is alive until query is finished. - std::vector> interpreter_context; - std::vector storage_holders; - /// If is set, all newly created processors will be added to this too. /// It is needed for debug. See QueryPipelineProcessorsCollector. Processors * collected_processors = nullptr; + /// This methods are for QueryPipeline. It is allowed to complete graph only there. + /// So, we may be sure that Pipe always has output port if not empty. + bool isCompleted() const { return !empty() && output_ports.empty(); } static Pipe unitePipes(Pipes pipes, Processors * collected_processors); + void setSinks(const Pipe::ProcessorGetterWithStreamKind & getter); friend class QueryPipeline; }; -/* -/// Pipe is a set of processors which represents the part of pipeline with single output. -/// All processors in pipe are connected. All ports are connected except the output one. -class Pipe -{ -public: - /// Create from source. It must have no input ports and single output. - explicit Pipe(ProcessorPtr source); - /// Connect several pipes together with specified transform. - /// Transform must have the number of inputs equals to the number of pipes. And single output. - /// Will connect pipes outputs with transform inputs automatically. - Pipe(Pipes && pipes, ProcessorPtr transform); - /// Create pipe from output port. If pipe was created that way, it possibly will not have tree shape. - explicit Pipe(OutputPort * port); - - Pipe(const Pipe & other) = delete; - Pipe(Pipe && other) = default; - - Pipe & operator=(const Pipe & other) = delete; - Pipe & operator=(Pipe && other) = default; - - /// Append processors to pipe. After this, it possibly will not have tree shape. - void addProcessors(const Processors & processors_); - - OutputPort & getPort() const { return *output_port; } - const Block & getHeader() const { return output_port->getHeader(); } - - /// Add transform to pipe. It must have single input and single output (is checked). - /// Input will be connected with current output port, output port will be updated. - void addSimpleTransform(ProcessorPtr transform); - - Processors detachProcessors() && { return std::move(processors); } - - /// Specify quotas and limits for every ISourceWithProgress. - void setLimits(const SourceWithProgress::LocalLimits & limits); - void setQuota(const std::shared_ptr & quota); - - /// Set information about preferred executor number for sources. - void pinSources(size_t executor_number); - - void enableQuota(); - - /// Totals and extremes port. - void setTotalsPort(OutputPort * totals_) { totals = totals_; } - void setExtremesPort(OutputPort * extremes_) { extremes = extremes_; } - OutputPort * getTotalsPort() const { return totals; } - OutputPort * getExtremesPort() const { return extremes; } - - size_t maxParallelStreams() const { return max_parallel_streams; } - - /// Do not allow to change the table while the processors of pipe are alive. - /// TODO: move it to pipeline. - void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } - /// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible. - void addInterpreterContext(std::shared_ptr context) { interpreter_context.emplace_back(std::move(context)); } - void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); } - - const std::vector & getTableLocks() const { return table_locks; } - const std::vector> & getContexts() const { return interpreter_context; } - const std::vector & getStorageHolders() const { return storage_holders; } - -private: - Processors processors; - OutputPort * output_port = nullptr; - OutputPort * totals = nullptr; - OutputPort * extremes = nullptr; - - /// It is the max number of processors which can be executed in parallel for each step. See QueryPipeline::Streams. - size_t max_parallel_streams = 0; - - std::vector table_locks; - - /// Some processors may implicitly use Context or temporary Storage created by Interpreter. - /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, - /// because QueryPipeline is alive until query is finished. - std::vector> interpreter_context; - std::vector storage_holders; - - /// This private constructor is used only from QueryPipeline. - /// It is not public, because QueryPipeline checks that processors are connected and have single output, - /// and therefore we can skip those checks. - /// Note that Pipe represents a tree if it was created using public interface. But this constructor can't assert it. - /// So, it's possible that TreeExecutorBlockInputStream could be unable to convert such Pipe to IBlockInputStream. - explicit Pipe(Processors processors_, OutputPort * output_port, OutputPort * totals, OutputPort * extremes); - - friend class QueryPipeline; -}; -*/ } diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 33d232da449..778d5ab093d 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -39,11 +40,11 @@ void QueryPipeline::checkInitializedAndNotCompleted() { checkInitialized(); - if (streams.empty()) + if (pipe.isCompleted()) throw Exception("QueryPipeline was already completed.", ErrorCodes::LOGICAL_ERROR); } -void QueryPipeline::checkSource(const ProcessorPtr & source, bool can_have_totals) +static void checkSource(const ProcessorPtr & source, bool can_have_totals) { if (!source->getInputs().empty()) throw Exception("Source for query pipeline shouldn't have any input, but " + source->getName() + " has " + @@ -62,289 +63,39 @@ void QueryPipeline::checkSource(const ProcessorPtr & source, bool can_have_total toString(source->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); } -void QueryPipeline::init(Pipe pipe) -{ - Pipes pipes; - pipes.emplace_back(std::move(pipe)); - init(std::move(pipes)); -} - -static OutputPort * uniteExtremes(const std::vector & ports, const Block & header, - QueryPipeline::ProcessorsContainer & processors) -{ - /// Here we calculate extremes for extremes in case we unite several pipelines. - /// Example: select number from numbers(2) union all select number from numbers(3) - - /// ->> Resize -> Extremes --(output port)----> Null - /// --(extremes port)--> ... - - auto resize = std::make_shared(header, ports.size(), 1); - auto extremes = std::make_shared(header); - auto sink = std::make_shared(header); - - auto * extremes_port = &extremes->getExtremesPort(); - - auto in = resize->getInputs().begin(); - for (const auto & port : ports) - connect(*port, *(in++)); - - connect(resize->getOutputs().front(), extremes->getInputPort()); - connect(extremes->getOutputPort(), sink->getPort()); - - processors.emplace(std::move(resize)); - processors.emplace(std::move(extremes)); - processors.emplace(std::move(sink)); - - return extremes_port; -} - -static OutputPort * uniteTotals(const std::vector & ports, const Block & header, - QueryPipeline::ProcessorsContainer & processors) -{ - /// Calculate totals fro several streams. - /// Take totals from first sources which has any, skip others. - - /// ->> Concat -> Limit - - auto concat = std::make_shared(header, ports.size()); - auto limit = std::make_shared(header, 1, 0); - - auto * totals_port = &limit->getOutputPort(); - - auto in = concat->getInputs().begin(); - for (const auto & port : ports) - connect(*port, *(in++)); - - connect(concat->getOutputs().front(), limit->getInputPort()); - - processors.emplace(std::move(concat)); - processors.emplace(std::move(limit)); - - return totals_port; -} - -void QueryPipeline::init(Pipe pipe) +void QueryPipeline::init(Pipe pipe_) { if (initialized()) throw Exception("Pipeline has already been initialized.", ErrorCodes::LOGICAL_ERROR); - if (pipes.empty()) - throw Exception("Can't initialize pipeline with empty pipes list.", ErrorCodes::LOGICAL_ERROR); - - /// Move locks from pipes to pipeline class. - for (auto & pipe : pipes) - { - for (const auto & lock : pipe.getTableLocks()) - table_locks.emplace_back(lock); - - for (const auto & context : pipe.getContexts()) - interpreter_context.emplace_back(context); - - for (const auto & storage : pipe.getStorageHolders()) - storage_holders.emplace_back(storage); - } - - std::vector totals; - std::vector extremes; - - for (auto & pipe : pipes) - { - const auto & header = pipe.getHeader(); - - if (current_header) - assertBlocksHaveEqualStructure(current_header, header, "QueryPipeline"); - else - current_header = header; - - if (auto * totals_port = pipe.getTotalsPort()) - { - assertBlocksHaveEqualStructure(current_header, totals_port->getHeader(), "QueryPipeline"); - totals.emplace_back(totals_port); - } - - if (auto * port = pipe.getExtremesPort()) - { - assertBlocksHaveEqualStructure(current_header, port->getHeader(), "QueryPipeline"); - extremes.emplace_back(port); - } - - streams.addStream(&pipe.getPort(), pipe.maxParallelStreams()); - processors.emplace(std::move(pipe).detachProcessors()); - } - - if (!totals.empty()) - { - if (totals.size() == 1) - totals_having_port = totals.back(); - else - totals_having_port = uniteTotals(totals, current_header, processors); - } - - if (!extremes.empty()) - { - if (extremes.size() == 1) - extremes_port = extremes.back(); - else - extremes_port = uniteExtremes(extremes, current_header, processors); - } -} - -static ProcessorPtr callProcessorGetter( - const Block & header, const QueryPipeline::ProcessorGetter & getter, QueryPipeline::StreamType) -{ - return getter(header); -} - -static ProcessorPtr callProcessorGetter( - const Block & header, const QueryPipeline::ProcessorGetterWithStreamKind & getter, QueryPipeline::StreamType kind) -{ - return getter(header, kind); -} - -template -void QueryPipeline::addSimpleTransformImpl(const TProcessorGetter & getter) -{ - checkInitializedAndNotCompleted(); - - Block header; - - auto add_transform = [&](OutputPort *& stream, StreamType stream_type) - { - if (!stream) - return; - - auto transform = callProcessorGetter(stream->getHeader(), getter, stream_type); - - if (transform) - { - if (transform->getInputs().size() != 1) - throw Exception("Processor for query pipeline transform should have single input, " - "but " + transform->getName() + " has " + - toString(transform->getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR); - - if (transform->getOutputs().size() != 1) - throw Exception("Processor for query pipeline transform should have single output, " - "but " + transform->getName() + " has " + - toString(transform->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); - } - - auto & out_header = transform ? transform->getOutputs().front().getHeader() - : stream->getHeader(); - - if (header) - assertBlocksHaveEqualStructure(header, out_header, "QueryPipeline"); - else - header = out_header; - - if (transform) - { - connect(*stream, transform->getInputs().front()); - stream = &transform->getOutputs().front(); - processors.emplace(std::move(transform)); - } - }; - - for (auto & stream : streams) - add_transform(stream, StreamType::Main); - - add_transform(totals_having_port, StreamType::Totals); - add_transform(extremes_port, StreamType::Extremes); - - current_header = std::move(header); -} - -void QueryPipeline::addSimpleTransform(const ProcessorGetter & getter) -{ - addSimpleTransformImpl(getter); -} - -void QueryPipeline::addSimpleTransform(const ProcessorGetterWithStreamKind & getter) -{ - addSimpleTransformImpl(getter); -} - -void QueryPipeline::setSinks(const ProcessorGetterWithStreamKind & getter) -{ - checkInitializedAndNotCompleted(); - - auto add_transform = [&](OutputPort *& stream, StreamType stream_type) - { - if (!stream) - return; - - auto transform = getter(stream->getHeader(), stream_type); - - if (transform) - { - if (transform->getInputs().size() != 1) - throw Exception("Sink for query pipeline transform should have single input, " - "but " + transform->getName() + " has " + - toString(transform->getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR); - - if (!transform->getOutputs().empty()) - throw Exception("Sink for query pipeline transform should have no outputs, " - "but " + transform->getName() + " has " + - toString(transform->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); - } - - if (!transform) - transform = std::make_shared(stream->getHeader()); - - connect(*stream, transform->getInputs().front()); - processors.emplace(std::move(transform)); - }; - - for (auto & stream : streams) - add_transform(stream, StreamType::Main); - - add_transform(totals_having_port, StreamType::Totals); - add_transform(extremes_port, StreamType::Extremes); - - streams.clear(); - current_header.clear(); -} - -void QueryPipeline::addPipe(Processors pipe) -{ - checkInitializedAndNotCompleted(); - if (pipe.empty()) - throw Exception("Can't add empty processors list to QueryPipeline.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Can't initialize pipeline with empty pipe.", ErrorCodes::LOGICAL_ERROR); - auto & first = pipe.front(); - auto & last = pipe.back(); + pipe = std::move(pipe_); +} - auto num_inputs = first->getInputs().size(); +void QueryPipeline::addSimpleTransform(const Pipe::ProcessorGetter & getter) +{ + checkInitializedAndNotCompleted(); + pipe.addSimpleTransform(getter); +} - if (num_inputs != streams.size()) - throw Exception("Can't add processors to QueryPipeline because first processor has " + toString(num_inputs) + - " input ports, but QueryPipeline has " + toString(streams.size()) + " streams.", - ErrorCodes::LOGICAL_ERROR); +void QueryPipeline::addSimpleTransform(const Pipe::ProcessorGetterWithStreamKind & getter) +{ + checkInitializedAndNotCompleted(); + pipe.addSimpleTransform(getter); +} - auto stream = streams.begin(); - for (auto & input : first->getInputs()) - connect(**(stream++), input); +void QueryPipeline::addTransform(ProcessorPtr transform) +{ + checkInitializedAndNotCompleted(); + pipe.addTransform(std::move(transform)); +} - Block header; - streams.clear(); - streams.reserve(last->getOutputs().size()); - for (auto & output : last->getOutputs()) - { - streams.addStream(&output, 0); - if (header) - assertBlocksHaveEqualStructure(header, output.getHeader(), "QueryPipeline"); - else - header = output.getHeader(); - } - - if (totals_having_port) - assertBlocksHaveEqualStructure(header, totals_having_port->getHeader(), "QueryPipeline"); - - if (extremes_port) - assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "QueryPipeline"); - - processors.emplace(pipe); - current_header = std::move(header); +void QueryPipeline::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) +{ + checkInitializedAndNotCompleted(); + pipe.setSinks(getter); } void QueryPipeline::addDelayedStream(ProcessorPtr source) @@ -352,14 +103,18 @@ void QueryPipeline::addDelayedStream(ProcessorPtr source) checkInitializedAndNotCompleted(); checkSource(source, false); - assertBlocksHaveEqualStructure(current_header, source->getOutputs().front().getHeader(), "QueryPipeline"); + assertBlocksHaveEqualStructure(getHeader(), source->getOutputs().front().getHeader(), "QueryPipeline"); - IProcessor::PortNumbers delayed_streams = { streams.size() }; - streams.addStream(&source->getOutputs().front(), 0); - processors.emplace(std::move(source)); + IProcessor::PortNumbers delayed_streams = { pipe.numOutputPorts() }; + pipe.addSource(std::move(source)); - auto processor = std::make_shared(current_header, streams.size(), delayed_streams); - addPipe({ std::move(processor) }); + auto processor = std::make_shared(getHeader(), pipe.numOutputPorts(), delayed_streams); + addTransform(std::move(processor)); +} + +void QueryPipeline::addMergingAggregatedMemoryEfficientTransform(AggregatingTransformParamsPtr params, size_t num_merging_processors) +{ + DB::addMergingAggregatedMemoryEfficientTransform(pipe, std::move(params), num_merging_processors); } void QueryPipeline::resize(size_t num_streams, bool force, bool strict) @@ -369,31 +124,14 @@ void QueryPipeline::resize(size_t num_streams, bool force, bool strict) if (!force && num_streams == getNumStreams()) return; - has_resize = true; - ProcessorPtr resize; if (strict) - resize = std::make_shared(current_header, getNumStreams(), num_streams); + resize = std::make_shared(getHeader(), getNumStreams(), num_streams); else - resize = std::make_shared(current_header, getNumStreams(), num_streams); + resize = std::make_shared(getHeader(), getNumStreams(), num_streams); - auto stream = streams.begin(); - for (auto & input : resize->getInputs()) - connect(**(stream++), input); - - streams.clear(); - streams.reserve(num_streams); - for (auto & output : resize->getOutputs()) - streams.addStream(&output, 0); - - processors.emplace(std::move(resize)); -} - -void QueryPipeline::enableQuotaForCurrentStreams() -{ - for (auto & stream : streams) - stream->getProcessor().enableQuota(); + pipe.addTransform(std::move(resize)); } void QueryPipeline::addTotalsHavingTransform(ProcessorPtr transform) @@ -404,28 +142,24 @@ void QueryPipeline::addTotalsHavingTransform(ProcessorPtr transform) throw Exception("TotalsHavingTransform expected for QueryPipeline::addTotalsHavingTransform.", ErrorCodes::LOGICAL_ERROR); - if (totals_having_port) + if (pipe.getTotalsPort()) throw Exception("Totals having transform was already added to pipeline.", ErrorCodes::LOGICAL_ERROR); resize(1); - connect(*streams.front(), transform->getInputs().front()); - - auto & outputs = transform->getOutputs(); - - streams.assign({ &outputs.front() }); - totals_having_port = &outputs.back(); - current_header = outputs.front().getHeader(); - processors.emplace(std::move(transform)); + auto * totals_port = &transform->getOutputs().back(); + pipe.addTransform(std::move(transform)); + pipe.totals_port = totals_port; } void QueryPipeline::addDefaultTotals() { checkInitializedAndNotCompleted(); - if (totals_having_port) + if (pipe.getTotalsPort()) throw Exception("Totals having transform was already added to pipeline.", ErrorCodes::LOGICAL_ERROR); + const auto & current_header = getHeader(); Columns columns; columns.reserve(current_header.columns()); @@ -437,22 +171,7 @@ void QueryPipeline::addDefaultTotals() } auto source = std::make_shared(current_header, Chunk(std::move(columns), 1)); - totals_having_port = &source->getPort(); - processors.emplace(std::move(source)); -} - -void QueryPipeline::addTotals(ProcessorPtr source) -{ - checkInitializedAndNotCompleted(); - - if (totals_having_port) - throw Exception("Totals having transform was already added to pipeline.", ErrorCodes::LOGICAL_ERROR); - - checkSource(source, false); - assertBlocksHaveEqualStructure(current_header, source->getOutputs().front().getHeader(), "QueryPipeline"); - - totals_having_port = &source->getOutputs().front(); - processors.emplace(std::move(source)); + pipe.addTotalsSource(std::move(source)); } void QueryPipeline::dropTotalsAndExtremes() @@ -837,8 +556,6 @@ QueryPipeline & QueryPipeline::operator= (QueryPipeline && rhs) rhs.max_threads = 0; output_format = rhs.output_format; rhs.output_format = nullptr; - has_resize = rhs.has_resize; - rhs.has_resize = false; extremes_port = rhs.extremes_port; rhs.extremes_port = nullptr; totals_having_port = rhs.totals_having_port; diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 5020a2e8964..61b4dc507fd 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -12,70 +12,37 @@ namespace DB { - -using TableLockHolders = std::vector; class Context; class IOutputFormat; class QueryPipelineProcessorsCollector; +struct AggregatingTransformParams; +using AggregatingTransformParamsPtr = std::shared_ptr; + class QueryPipeline { public: - - class ProcessorsContainer - { - public: - bool empty() const { return processors.empty(); } - void emplace(ProcessorPtr processor); - void emplace(Processors processors_); - Processors * getCollectedProcessors() const { return collected_processors; } - Processors * setCollectedProcessors(Processors * collected_processors); - Processors & get() { return processors; } - const Processors & get() const { return processors; } - Processors detach() { return std::move(processors); } - private: - /// All added processors. - Processors processors; - - /// If is set, all newly created processors will be added to this too. - /// It is needed for debug. See QueryPipelineProcessorsCollector below. - Processors * collected_processors = nullptr; - }; - QueryPipeline() = default; - QueryPipeline(QueryPipeline &&) = default; ~QueryPipeline() = default; + QueryPipeline(QueryPipeline &&) = default; QueryPipeline(const QueryPipeline &) = delete; - QueryPipeline & operator= (const QueryPipeline & rhs) = delete; - QueryPipeline & operator= (QueryPipeline && rhs); + QueryPipeline & operator= (const QueryPipeline & rhs) = delete; /// All pipes must have same header. void init(Pipe pipe); - bool initialized() { return !processors.empty(); } - bool isCompleted() { return initialized() && streams.empty(); } + bool initialized() { return !pipe.empty(); } + bool isCompleted() { return pipe.isCompleted(); } - /// Type of logical data stream for simple transform. - /// Sometimes it's important to know which part of pipeline we are working for. - /// Example: ExpressionTransform need special logic for totals. - enum class StreamType - { - Main = 0, /// Stream for query data. There may be several streams of this type. - Totals, /// Stream for totals. No more then one. - Extremes, /// Stream for extremes. No more then one. - }; - - using ProcessorGetter = std::function; - using ProcessorGetterWithStreamKind = std::function; + using StreamType = Pipe::StreamType; /// Add transform with simple input and simple output for each port. - void addSimpleTransform(const ProcessorGetter & getter); - void addSimpleTransform(const ProcessorGetterWithStreamKind & getter); - /// Add several processors. They must have same header for inputs and same for outputs. - /// Total number of inputs must be the same as the number of streams. Output ports will become new streams. - void addPipe(Processors pipe); + void addSimpleTransform(const Pipe::ProcessorGetter & getter); + void addSimpleTransform(const Pipe::ProcessorGetterWithStreamKind & getter); + /// Add transform with getNumStreams() input ports. + void addTransform(ProcessorPtr transform); /// Add TotalsHavingTransform. Resize pipeline to single input. Adds totals port. void addTotalsHavingTransform(ProcessorPtr transform); /// Add transform which calculates extremes. This transform adds extremes port and doesn't change inputs number. @@ -88,40 +55,37 @@ public: IOutputFormat * getOutputFormat() const { return output_format; } /// Sink is a processor with single input port and no output ports. Creates sink for each output port. /// Pipeline will be completed after this transformation. - void setSinks(const ProcessorGetterWithStreamKind & getter); + void setSinks(const Pipe::ProcessorGetterWithStreamKind & getter); /// Add totals which returns one chunk with single row with defaults. void addDefaultTotals(); - /// Add already calculated totals. - void addTotals(ProcessorPtr source); - /// Forget about current totals and extremes. It is needed before aggregation, cause they will be calculated again. void dropTotalsAndExtremes(); /// Will read from this stream after all data was read from other streams. void addDelayedStream(ProcessorPtr source); + void addMergingAggregatedMemoryEfficientTransform(AggregatingTransformParamsPtr params, size_t num_merging_processors); + /// Changes the number of input ports if needed. Adds ResizeTransform. void resize(size_t num_streams, bool force = false, bool strict = false); - void enableQuotaForCurrentStreams(); - /// Unite several pipelines together. Result pipeline would have common_header structure. /// If collector is used, it will collect only newly-added processors, but not processors from pipelines. void unitePipelines(std::vector> pipelines, const Block & common_header, size_t max_threads_limit = 0); PipelineExecutorPtr execute(); - size_t getNumStreams() const { return streams.size(); } + size_t getNumStreams() const { return pipe.numOutputPorts(); } - bool hasTotals() const { return totals_having_port != nullptr; } + bool hasTotals() const { return pipe.getTotalsPort() != nullptr; } - const Block & getHeader() const { return current_header; } + const Block & getHeader() const { return pipe.getHeader(); } - void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } - void addInterpreterContext(std::shared_ptr context) { interpreter_context.emplace_back(std::move(context)); } - void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); } + void addTableLock(const TableLockHolder & lock) { pipe.addTableLock(lock); } + void addInterpreterContext(std::shared_ptr context) { pipe.addInterpreterContext(std::move(context)); } + void addStorageHolder(StoragePtr storage) { pipe.addStorageHolder(std::move(storage)); } /// For compatibility with IBlockInputStream. void setProgressCallback(const ProgressCallback & callback); @@ -130,7 +94,7 @@ public: /// Recommend number of threads for pipeline execution. size_t getNumThreads() const { - auto num_threads = streams.maxParallelStreams(); + auto num_threads = pipe.maxParallelStreams(); if (max_threads) num_threads = std::min(num_threads, max_threads); @@ -151,34 +115,10 @@ public: /// Convert query pipeline to pipe. Pipe getPipe() &&; - /// Get internal processors. - const Processors & getProcessors() const { return processors.get(); } - private: /// Destruction order: processors, header, locks, temporary storages, local contexts - /// Some Streams (or Processors) may implicitly use Context or temporary Storage created by Interpreter. - /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, - /// because QueryPipeline is alive until query is finished. - std::vector> interpreter_context; - std::vector storage_holders; - TableLockHolders table_locks; - - /// Common header for each stream. - Block current_header; - - ProcessorsContainer processors; - - /// Port for each independent "stream". - Streams streams; - - /// Special ports for extremes and totals having. - OutputPort * totals_having_port = nullptr; - OutputPort * extremes_port = nullptr; - - /// If resize processor was added to pipeline. - bool has_resize = false; - + Pipe pipe; IOutputFormat * output_format = nullptr; /// Limit on the number of threads. Zero means no limit. @@ -189,10 +129,6 @@ private: void checkInitialized(); void checkInitializedAndNotCompleted(); - static void checkSource(const ProcessorPtr & source, bool can_have_totals); - - template - void addSimpleTransformImpl(const TProcessorGetter & getter); void initRowsBeforeLimit(); diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index a78c813ef8a..e8d4a262366 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -101,7 +101,7 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline) group_by_sort_description, max_block_size); - pipeline.addPipe({ std::move(transform) }); + pipeline.addTransform(std::move(transform)); aggregating_sorted = collector.detachProcessors(1); } else @@ -120,8 +120,6 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline) }); finalizing = collector.detachProcessors(2); - - pipeline.enableQuotaForCurrentStreams(); return; } } @@ -156,8 +154,6 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline) aggregating = collector.detachProcessors(0); } - - pipeline.enableQuotaForCurrentStreams(); } void AggregatingStep::describeActions(FormatSettings & settings) const diff --git a/src/Processors/QueryPlan/FinishSortingStep.cpp b/src/Processors/QueryPlan/FinishSortingStep.cpp index 8d9db8f83a1..d883bd0e0dd 100644 --- a/src/Processors/QueryPlan/FinishSortingStep.cpp +++ b/src/Processors/QueryPlan/FinishSortingStep.cpp @@ -63,11 +63,9 @@ void FinishSortingStep::transformPipeline(QueryPipeline & pipeline) prefix_description, max_block_size, limit_for_merging); - pipeline.addPipe({ std::move(transform) }); + pipeline.addTransform(std::move(transform)); } - pipeline.enableQuotaForCurrentStreams(); - if (need_finish_sorting) { pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr diff --git a/src/Processors/QueryPlan/LimitStep.cpp b/src/Processors/QueryPlan/LimitStep.cpp index 21ee1462ef5..565d05956e5 100644 --- a/src/Processors/QueryPlan/LimitStep.cpp +++ b/src/Processors/QueryPlan/LimitStep.cpp @@ -47,7 +47,7 @@ void LimitStep::transformPipeline(QueryPipeline & pipeline) auto transform = std::make_shared( pipeline.getHeader(), limit, offset, pipeline.getNumStreams(), always_read_till_end, with_ties, description); - pipeline.addPipe({std::move(transform)}); + pipeline.addTransform(std::move(transform)); } void LimitStep::describeActions(FormatSettings & settings) const diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index 473cfb0d125..85bbbeab59a 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -59,16 +59,8 @@ void MergingAggregatedStep::transformPipeline(QueryPipeline & pipeline) ? static_cast(memory_efficient_merge_threads) : static_cast(max_threads); - auto pipe = createMergingAggregatedMemoryEfficientPipe( - pipeline.getHeader(), - params, - pipeline.getNumStreams(), - num_merge_threads); - - pipeline.addPipe(std::move(pipe)); + pipeline.addMergingAggregatedMemoryEfficientTransform(params, num_merge_threads); } - - pipeline.enableQuotaForCurrentStreams(); } void MergingAggregatedStep::describeActions(FormatSettings & settings) const diff --git a/src/Processors/QueryPlan/MergingSortedStep.cpp b/src/Processors/QueryPlan/MergingSortedStep.cpp index 98a7deebc9e..c59540b009f 100644 --- a/src/Processors/QueryPlan/MergingSortedStep.cpp +++ b/src/Processors/QueryPlan/MergingSortedStep.cpp @@ -58,9 +58,7 @@ void MergingSortedStep::transformPipeline(QueryPipeline & pipeline) sort_description, max_block_size, limit); - pipeline.addPipe({ std::move(transform) }); - - pipeline.enableQuotaForCurrentStreams(); + pipeline.addTransform(std::move(transform)); } } diff --git a/src/Processors/QueryPlan/OffsetStep.cpp b/src/Processors/QueryPlan/OffsetStep.cpp index 66f2ada2152..7ac3d3f2110 100644 --- a/src/Processors/QueryPlan/OffsetStep.cpp +++ b/src/Processors/QueryPlan/OffsetStep.cpp @@ -33,7 +33,7 @@ void OffsetStep::transformPipeline(QueryPipeline & pipeline) auto transform = std::make_shared( pipeline.getHeader(), offset, pipeline.getNumStreams()); - pipeline.addPipe({std::move(transform)}); + pipeline.addTransform(std::move(transform)); } void OffsetStep::describeActions(FormatSettings & settings) const diff --git a/src/Processors/Sources/DelayedSource.cpp b/src/Processors/Sources/DelayedSource.cpp index f9e112f5e68..f0cc6b1930c 100644 --- a/src/Processors/Sources/DelayedSource.cpp +++ b/src/Processors/Sources/DelayedSource.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -111,15 +112,27 @@ void synchronizePorts(OutputPort *& pipe_port, OutputPort * source_port, const B void DelayedSource::work() { auto pipe = creator(); + const auto & header = main->getHeader(); - main_output = &pipe.getPort(); + if (pipe.empty()) + { + auto source = std::make_shared(header); + main_output = &source->getPort(); + processors.emplace_back(std::move(source)); + return; + } + + if (pipe.numOutputPorts() > 1) + pipe.addTransform(std::make_shared(header, pipe.numOutputPorts(), 1)); + + main_output = pipe.getOutputPort(0); totals_output = pipe.getTotalsPort(); extremes_output = pipe.getExtremesPort(); - processors = std::move(pipe).detachProcessors(); + processors = Pipe::detachProcessors(std::move(pipe)); - synchronizePorts(totals_output, totals, main->getHeader(), processors); - synchronizePorts(extremes_output, extremes, main->getHeader(), processors); + synchronizePorts(totals_output, totals, header, processors); + synchronizePorts(extremes_output, extremes, header, processors); } Processors DelayedSource::expandPipeline() @@ -144,12 +157,11 @@ Pipe createDelayedPipe(const Block & header, DelayedSource::Creator processors_c { auto source = std::make_shared(header, std::move(processors_creator), add_totals_port, add_extremes_port); - Pipe pipe(&source->getPort()); - pipe.setTotalsPort(source->getTotalsPort()); - pipe.setExtremesPort(source->getExtremesPort()); + auto * main = &source->getPort(); + auto * totals = source->getTotalsPort(); + auto * extremes = source->getExtremesPort(); - pipe.addProcessors({std::move(source)}); - return pipe; + return Pipe(std::move(source), main, totals, extremes); } } diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index c5be62e276a..8fb6f13a85b 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -585,23 +586,24 @@ void AggregatingTransform::initGenerate() } } - auto header = params->aggregator.getHeader(false); - const auto & files = params->aggregator.getTemporaryFiles(); - BlockInputStreams input_streams; - for (const auto & file : files.files) - processors.emplace_back(std::make_unique(header, file->path())); + Pipe pipe; + + { + auto header = params->aggregator.getHeader(false); + Pipes pipes; + + for (const auto & file : files.files) + pipes.emplace_back(Pipe(std::make_unique(header, file->path()))); + + pipe = Pipe::unitePipes(std::move(pipes)); + } LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), ReadableSize(files.sum_size_compressed), ReadableSize(files.sum_size_uncompressed)); - auto pipe = createMergingAggregatedMemoryEfficientPipe( - header, params, files.files.size(), temporary_data_merge_threads); + addMergingAggregatedMemoryEfficientTransform(pipe, params, temporary_data_merge_threads); - auto input = pipe.front()->getInputs().begin(); - for (auto & processor : processors) - connect(processor->getOutputs().front(), *(input++)); - - processors.insert(processors.end(), pipe.begin(), pipe.end()); + processors = Pipe::detachProcessors(std::move(pipe)); } } diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp index 6496c32738e..3445b1b3312 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp @@ -1,8 +1,9 @@ #include -#include #include +#include #include +#include namespace DB { @@ -492,51 +493,32 @@ IProcessor::Status SortingAggregatedTransform::prepare() } -Processors createMergingAggregatedMemoryEfficientPipe( - Block header, - AggregatingTransformParamsPtr params, - size_t num_inputs, - size_t num_merging_processors) +void addMergingAggregatedMemoryEfficientTransform( + Pipe & pipe, + AggregatingTransformParamsPtr params, + size_t num_merging_processors) { - Processors processors; - processors.reserve(num_merging_processors + 2); - - auto grouping = std::make_shared(header, num_inputs, params); - processors.emplace_back(std::move(grouping)); + pipe.addTransform(std::make_shared(pipe.getHeader(), pipe.numOutputPorts(), params)); if (num_merging_processors <= 1) { /// --> GroupingAggregated --> MergingAggregatedBucket --> - auto transform = std::make_shared(params); - connect(processors.back()->getOutputs().front(), transform->getInputPort()); - - processors.emplace_back(std::move(transform)); - return processors; + pipe.addTransform(std::make_shared(params)); + return; } /// --> --> MergingAggregatedBucket --> /// --> GroupingAggregated --> ResizeProcessor --> MergingAggregatedBucket --> SortingAggregated --> /// --> --> MergingAggregatedBucket --> - auto resize = std::make_shared(Block(), 1, num_merging_processors); - connect(processors.back()->getOutputs().front(), resize->getInputs().front()); - processors.emplace_back(std::move(resize)); + pipe.addTransform(std::make_shared(Block(), 1, num_merging_processors)); - auto sorting = std::make_shared(num_merging_processors, params); - auto out = processors.back()->getOutputs().begin(); - auto in = sorting->getInputs().begin(); - - for (size_t i = 0; i < num_merging_processors; ++i, ++in, ++out) + pipe.addSimpleTransform([params](const Block &) { - auto transform = std::make_shared(params); - transform->setStream(i); - connect(*out, transform->getInputPort()); - connect(transform->getOutputPort(), *in); - processors.emplace_back(std::move(transform)); - } + return std::make_shared(params); + }); - processors.emplace_back(std::move(sorting)); - return processors; + pipe.addTransform(std::make_shared(num_merging_processors, params)); } } diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h index 1ed1979c0d4..1f0b7ed4157 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h @@ -136,12 +136,12 @@ private: void addChunk(Chunk chunk, size_t from_input); }; -/// Creates piece of pipeline which performs memory efficient merging of partially aggregated data from several sources. -/// First processor will have num_inputs, last - single output. You should connect them to create pipeline. -Processors createMergingAggregatedMemoryEfficientPipe( - Block header, +class Pipe; + +/// Adds processors to pipe which performs memory efficient merging of partially aggregated data from several sources. +void addMergingAggregatedMemoryEfficientTransform( + Pipe & pipe, AggregatingTransformParamsPtr params, - size_t num_inputs, size_t num_merging_processors); } From a153f05e10fe9ddfb7607672385c8ce6a6a826dc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Aug 2020 18:51:56 +0300 Subject: [PATCH 122/374] Refactor Pipe [part 5]. --- src/Interpreters/InterpreterAlterQuery.cpp | 4 +- src/Processors/Pipe.cpp | 119 ++++++++- src/Processors/Pipe.h | 7 +- src/Processors/QueryPipeline.cpp | 269 ++++----------------- src/Processors/QueryPipeline.h | 13 +- 5 files changed, 164 insertions(+), 248 deletions(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 2f9e93d0eee..6f1215d6af0 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -87,8 +87,8 @@ BlockIO InterpreterAlterQuery::execute() if (!partition_commands.empty()) { table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, context.getSettingsRef()); - auto partition_commands_pipes = table->alterPartition(query_ptr, metadata_snapshot, partition_commands, context); - if (!partition_commands_pipes.empty()) + auto partition_commands_pipe = table->alterPartition(query_ptr, metadata_snapshot, partition_commands, context); + if (!partition_commands_pipe.empty()) res.pipeline.init(std::move(partition_commands_pipes)); } diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index 65c2e5f2a5a..7a0c7184882 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include namespace DB { @@ -374,7 +376,37 @@ void Pipe::addExtremesSource(ProcessorPtr source) processors.emplace_back(std::move(source)); } +static void dropPort(OutputPort *& port, Processors & processors, Processors * collected_processors) +{ + if (port == nullptr) + return; + + auto null_sink = std::make_shared(port->getHeader()); + connect(*port, null_sink->getPort()); + + if (collected_processors) + collected_processors->emplace_back(null_sink.get()); + + processors.emplace_back(std::move(null_sink)); + port = nullptr; +} + +void Pipe::dropTotals() +{ + dropPort(totals_port, processors, collected_processors); +} + +void Pipe::dropExtremes() +{ + dropPort(extremes_port, processors, collected_processors); +} + void Pipe::addTransform(ProcessorPtr transform) +{ + addTransform(std::move(transform), nullptr, nullptr); +} + +void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort * extremes) { if (output_ports.empty()) throw Exception("Cannot add transform to empty Pipe.", ErrorCodes::LOGICAL_ERROR); @@ -385,6 +417,19 @@ void Pipe::addTransform(ProcessorPtr transform) "Processor has " + std::to_string(inputs.size()) + " input ports, " "but " + std::to_string(output_ports.size()) + " expected", ErrorCodes::LOGICAL_ERROR); + if (totals && totals_port) + throw Exception("Cannot add transform with totals to Pipe because it already has totals.", + ErrorCodes::LOGICAL_ERROR); + + if (extremes && extremes_port) + throw Exception("Cannot add transform with totals to Pipe because it already has totals.", + ErrorCodes::LOGICAL_ERROR); + + if (totals) + totals_port = totals; + if (extremes) + extremes_port = extremes; + size_t next_output = 0; for (auto & input : inputs) { @@ -393,15 +438,34 @@ void Pipe::addTransform(ProcessorPtr transform) } auto & outputs = transform->getOutputs(); - if (outputs.empty()) - throw Exception("Cannot add transform " + transform->getName() + " to Pipes because it has no outputs", - ErrorCodes::LOGICAL_ERROR); output_ports.clear(); output_ports.reserve(outputs.size()); + bool found_totals = false; + bool found_extremes = false; + for (auto & output : outputs) - output_ports.emplace_back(&output); + { + if (&output == totals) + found_totals = true; + else if (&output == extremes) + found_extremes = true; + else + output_ports.emplace_back(&output); + } + + if (totals && !found_totals) + throw Exception("Cannot add transform " + transform->getName() + " to Pipes because " + "specified totals port does not belong to it", ErrorCodes::LOGICAL_ERROR); + + if (extremes && !found_extremes) + throw Exception("Cannot add transform " + transform->getName() + " to Pipes because " + "specified extremes port does not belong to it", ErrorCodes::LOGICAL_ERROR); + + if (output_ports.empty()) + throw Exception("Cannot add transform " + transform->getName() + " to Pipes because it has no outputs", + ErrorCodes::LOGICAL_ERROR); header = output_ports.front()->getHeader(); for (size_t i = 1; i < output_ports.size(); ++i) @@ -524,6 +588,44 @@ void Pipe::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) header.clear(); } +void Pipe::setOutputFormat(ProcessorPtr output) +{ + if (output_ports.empty()) + throw Exception("Cannot set output format to empty Pipe.", ErrorCodes::LOGICAL_ERROR); + + if (output_ports.size() != 1) + throw Exception("Cannot set output format to Pipe because single output port is expected, " + "but it has " + std::to_string(output_ports.size()) + " ports", ErrorCodes::LOGICAL_ERROR); + + auto * format = dynamic_cast(output.get()); + + if (!format) + throw Exception("IOutputFormat processor expected for QueryPipeline::setOutputFormat.", + ErrorCodes::LOGICAL_ERROR); + + auto & main = format->getPort(IOutputFormat::PortKind::Main); + auto & totals = format->getPort(IOutputFormat::PortKind::Totals); + auto & extremes = format->getPort(IOutputFormat::PortKind::Extremes); + + if (!totals_port) + addTotalsSource(std::make_shared(totals.getHeader())); + + if (!extremes_port) + addExtremesSource(std::make_shared(extremes.getHeader())); + + if (collected_processors) + collected_processors->emplace_back(output.get()); + + processors.emplace_back(std::move(output)); + + connect(*output_ports.front(), main); + connect(*totals_port, totals); + connect(*extremes_port, extremes); + + output_ports.clear(); + header.clear(); +} + void Pipe::transform(const Transformer & transformer) { if (output_ports.empty()) @@ -620,13 +722,4 @@ void Pipe::setQuota(const std::shared_ptr & quota) } } -void Pipe::enableQuota() -{ - for (auto & processor : processors) - { - if (auto * source = dynamic_cast(processor.get())) - source->enableQuota(); - } -} - } diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index e6ae375da48..787bade065a 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -52,10 +52,15 @@ public: void addTotalsSource(ProcessorPtr source); void addExtremesSource(ProcessorPtr source); + /// Drop totals and extremes (create NullSink for them). + void dropTotals(); + void dropExtremes(); + /// Add processor to list. It should have size() input ports with compatible header. /// Output ports should have same headers. /// If totals or extremes are not empty, transform shouldn't change header. void addTransform(ProcessorPtr transform); + void addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort * extremes); enum class StreamType { @@ -85,7 +90,6 @@ public: /// Specify quotas and limits for every ISourceWithProgress. void setLimits(const SourceWithProgress::LocalLimits & limits); void setQuota(const std::shared_ptr & quota); - void enableQuota(); /// Do not allow to change the table while the processors of pipe are alive. void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } @@ -125,6 +129,7 @@ private: bool isCompleted() const { return !empty() && output_ports.empty(); } static Pipe unitePipes(Pipes pipes, Processors * collected_processors); void setSinks(const Pipe::ProcessorGetterWithStreamKind & getter); + void setOutputFormat(ProcessorPtr output); friend class QueryPipeline; }; diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 778d5ab093d..97e96387f7f 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -2,9 +2,7 @@ #include #include -#include #include -#include #include #include #include @@ -148,8 +146,7 @@ void QueryPipeline::addTotalsHavingTransform(ProcessorPtr transform) resize(1); auto * totals_port = &transform->getOutputs().back(); - pipe.addTransform(std::move(transform)); - pipe.totals_port = totals_port; + pipe.addTransform(std::move(transform), totals_port, nullptr); } void QueryPipeline::addDefaultTotals() @@ -176,46 +173,21 @@ void QueryPipeline::addDefaultTotals() void QueryPipeline::dropTotalsAndExtremes() { - auto drop_port = [&](OutputPort *& port) - { - auto null_sink = std::make_shared(port->getHeader()); - connect(*port, null_sink->getPort()); - processors.emplace(std::move(null_sink)); - port = nullptr; - }; - - if (totals_having_port) - drop_port(totals_having_port); - - if (extremes_port) - drop_port(extremes_port); + pipe.dropTotals(); + pipe.dropExtremes(); } void QueryPipeline::addExtremesTransform() { checkInitializedAndNotCompleted(); - if (extremes_port) + if (pipe.getExtremesPort()) throw Exception("Extremes transform was already added to pipeline.", ErrorCodes::LOGICAL_ERROR); - std::vector extremes; - extremes.reserve(streams.size()); - - for (auto & stream : streams) - { - auto transform = std::make_shared(current_header); - connect(*stream, transform->getInputPort()); - - stream = &transform->getOutputPort(); - extremes.push_back(&transform->getExtremesPort()); - - processors.emplace(std::move(transform)); - } - - if (extremes.size() == 1) - extremes_port = extremes.front(); - else - extremes_port = uniteExtremes(extremes, current_header, processors); + resize(1); + auto transform = std::make_shared(getHeader()); + auto * port = &transform->getExtremesPort(); + pipe.addTransform(std::move(transform), nullptr, port); } void QueryPipeline::addCreatingSetsTransform(ProcessorPtr transform) @@ -228,94 +200,49 @@ void QueryPipeline::addCreatingSetsTransform(ProcessorPtr transform) resize(1); - auto concat = std::make_shared(current_header, 2); - connect(transform->getOutputs().front(), concat->getInputs().front()); - connect(*streams.back(), concat->getInputs().back()); - - streams.assign({ &concat->getOutputs().front() }); - processors.emplace(std::move(transform)); - processors.emplace(std::move(concat)); + /// Order is important for concat. Connect manually. + pipe.transform([&](OutputPortRawPtrs ports) -> Processors + { + auto concat = std::make_shared(getHeader(), 2); + connect(transform->getOutputs().front(), concat->getInputs().front()); + connect(*ports.back(), concat->getInputs().back()); + return { std::move(concat), std::move(transform) }; + }); } void QueryPipeline::setOutputFormat(ProcessorPtr output) { checkInitializedAndNotCompleted(); - auto * format = dynamic_cast(output.get()); - - if (!format) - throw Exception("IOutputFormat processor expected for QueryPipeline::setOutputFormat.", ErrorCodes::LOGICAL_ERROR); - if (output_format) throw Exception("QueryPipeline already has output.", ErrorCodes::LOGICAL_ERROR); - output_format = format; - resize(1); - auto & main = format->getPort(IOutputFormat::PortKind::Main); - auto & totals = format->getPort(IOutputFormat::PortKind::Totals); - auto & extremes = format->getPort(IOutputFormat::PortKind::Extremes); - - if (!totals_having_port) - { - auto null_source = std::make_shared(totals.getHeader()); - totals_having_port = &null_source->getPort(); - processors.emplace(std::move(null_source)); - } - - if (!extremes_port) - { - auto null_source = std::make_shared(extremes.getHeader()); - extremes_port = &null_source->getPort(); - processors.emplace(std::move(null_source)); - } - - processors.emplace(std::move(output)); - - connect(*streams.front(), main); - connect(*totals_having_port, totals); - connect(*extremes_port, extremes); - - streams.clear(); - current_header.clear(); - extremes_port = nullptr; - totals_having_port = nullptr; + output_format = dynamic_cast(output.get()); + pipe.setOutputFormat(std::move(output)); initRowsBeforeLimit(); } -void QueryPipeline::unitePipelines( - std::vector> pipelines, const Block & common_header, size_t max_threads_limit) +QueryPipeline QueryPipeline::unitePipelines( + std::vector> pipelines, + const Block & common_header, + size_t max_threads_limit, + Processors * collected_processors) { /// Should we limit the number of threads for united pipeline. True if all pipelines have max_threads != 0. /// If true, result max_threads will be sum(max_threads). /// Note: it may be > than settings.max_threads, so we should apply this limit again. - bool will_limit_max_threads = !initialized() || max_threads != 0; - - if (initialized()) - { - addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, common_header, ConvertingTransform::MatchColumnsMode::Position); - }); - } - - std::vector extremes; - std::vector totals; - - if (extremes_port) - extremes.push_back(extremes_port); - - if (totals_having_port) - totals.push_back(totals_having_port); + bool will_limit_max_threads = true; + size_t max_threads = 0; + Pipes pipes; for (auto & pipeline_ptr : pipelines) { auto & pipeline = *pipeline_ptr; pipeline.checkInitialized(); - pipeline.processors.setCollectedProcessors(processors.getCollectedProcessors()); + pipeline.pipe.collected_processors = collected_processors; if (!pipeline.isCompleted()) { @@ -326,36 +253,7 @@ void QueryPipeline::unitePipelines( }); } - if (pipeline.extremes_port) - { - auto converting = std::make_shared( - pipeline.current_header, common_header, ConvertingTransform::MatchColumnsMode::Position); - - connect(*pipeline.extremes_port, converting->getInputPort()); - extremes.push_back(&converting->getOutputPort()); - processors.emplace(std::move(converting)); - } - - /// Take totals only from first port. - if (pipeline.totals_having_port) - { - auto converting = std::make_shared( - pipeline.current_header, common_header, ConvertingTransform::MatchColumnsMode::Position); - - connect(*pipeline.totals_having_port, converting->getInputPort()); - totals.push_back(&converting->getOutputPort()); - processors.emplace(std::move(converting)); - } - - auto * collector = processors.setCollectedProcessors(nullptr); - processors.emplace(pipeline.processors.detach()); - processors.setCollectedProcessors(collector); - - streams.addStreams(pipeline.streams); - - table_locks.insert(table_locks.end(), std::make_move_iterator(pipeline.table_locks.begin()), std::make_move_iterator(pipeline.table_locks.end())); - interpreter_context.insert(interpreter_context.end(), pipeline.interpreter_context.begin(), pipeline.interpreter_context.end()); - storage_holders.insert(storage_holders.end(), pipeline.storage_holders.begin(), pipeline.storage_holders.end()); + pipes.emplace_back(std::move(pipeline.pipe)); max_threads += pipeline.max_threads; will_limit_max_threads = will_limit_max_threads && pipeline.max_threads != 0; @@ -366,33 +264,21 @@ void QueryPipeline::unitePipelines( max_threads_limit = pipeline.max_threads; } - if (!will_limit_max_threads) - max_threads = 0; - else - limitMaxThreads(max_threads_limit); + QueryPipeline pipeline; + pipeline.init(Pipe::unitePipes(std::move(pipes), collected_processors)); - if (!extremes.empty()) + if (will_limit_max_threads) { - if (extremes.size() == 1) - extremes_port = extremes.back(); - else - extremes_port = uniteExtremes(extremes, common_header, processors); + pipeline.setMaxThreads(max_threads); + pipeline.limitMaxThreads(max_threads_limit); } - if (!totals.empty()) - { - if (totals.size() == 1) - totals_having_port = totals.back(); - else - totals_having_port = uniteTotals(totals, common_header, processors); - } - - current_header = common_header; + return pipeline; } void QueryPipeline::setProgressCallback(const ProgressCallback & callback) { - for (auto & processor : processors.get()) + for (auto & processor : pipe.processors) { if (auto * source = dynamic_cast(processor.get())) source->setProgressCallback(callback); @@ -406,7 +292,7 @@ void QueryPipeline::setProcessListElement(QueryStatus * elem) { process_list_element = elem; - for (auto & processor : processors.get()) + for (auto & processor : pipe.processors) { if (auto * source = dynamic_cast(processor.get())) source->setProcessListElement(elem); @@ -510,101 +396,28 @@ void QueryPipeline::initRowsBeforeLimit() output_format->setRowsBeforeLimitCounter(rows_before_limit_at_least); } -Pipe QueryPipeline::getPipe() && -{ - Pipes pipes(processors.detach(), streams.at(0), totals_having_port, extremes_port); - pipe.max_parallel_streams = streams.maxParallelStreams(); - - for (auto & lock : table_locks) - pipe.addTableLock(lock); - - for (auto & context : interpreter_context) - pipe.addInterpreterContext(context); - - for (auto & storage : storage_holders) - pipe.addStorageHolder(storage); - - if (totals_having_port) - pipe.setTotalsPort(totals_having_port); - - if (extremes_port) - pipe.setExtremesPort(extremes_port); - - Pipes pipes; - pipes.emplace_back(std::move(pipe)); - - for (size_t i = 1; i < streams.size(); ++i) - pipes.emplace_back(Pipe(streams[i])); - - return pipes; -} - PipelineExecutorPtr QueryPipeline::execute() { if (!isCompleted()) throw Exception("Cannot execute pipeline because it is not completed.", ErrorCodes::LOGICAL_ERROR); - return std::make_shared(processors.get(), process_list_element); + return std::make_shared(pipe.processors, process_list_element); } -QueryPipeline & QueryPipeline::operator= (QueryPipeline && rhs) +void QueryPipeline::setCollectedProcessors(Processors * processors) { - /// Reset primitive fields - process_list_element = rhs.process_list_element; - rhs.process_list_element = nullptr; - max_threads = rhs.max_threads; - rhs.max_threads = 0; - output_format = rhs.output_format; - rhs.output_format = nullptr; - extremes_port = rhs.extremes_port; - rhs.extremes_port = nullptr; - totals_having_port = rhs.totals_having_port; - rhs.totals_having_port = nullptr; - - /// Move these fields in destruction order (it's important) - streams = std::move(rhs.streams); - processors = std::move(rhs.processors); - current_header = std::move(rhs.current_header); - table_locks = std::move(rhs.table_locks); - storage_holders = std::move(rhs.storage_holders); - interpreter_context = std::move(rhs.interpreter_context); - - return *this; -} - -void QueryPipeline::ProcessorsContainer::emplace(ProcessorPtr processor) -{ - if (collected_processors) - collected_processors->emplace_back(processor); - - processors.emplace_back(std::move(processor)); -} - -void QueryPipeline::ProcessorsContainer::emplace(Processors processors_) -{ - for (auto & processor : processors_) - emplace(std::move(processor)); -} - -Processors * QueryPipeline::ProcessorsContainer::setCollectedProcessors(Processors * collected_processors_) -{ - if (collected_processors && collected_processors_) - throw Exception("Cannot set collected processors to QueryPipeline because " - "another one object was already created for current pipeline." , ErrorCodes::LOGICAL_ERROR); - - std::swap(collected_processors, collected_processors_); - return collected_processors_; + pipe.collected_processors = processors; } QueryPipelineProcessorsCollector::QueryPipelineProcessorsCollector(QueryPipeline & pipeline_, IQueryPlanStep * step_) : pipeline(pipeline_), step(step_) { - pipeline.processors.setCollectedProcessors(&processors); + pipeline.setCollectedProcessors(&processors); } QueryPipelineProcessorsCollector::~QueryPipelineProcessorsCollector() { - pipeline.processors.setCollectedProcessors(nullptr); + pipeline.setCollectedProcessors(nullptr); } Processors QueryPipelineProcessorsCollector::detachProcessors(size_t group) diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 61b4dc507fd..74cc692ac42 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -28,7 +28,7 @@ public: ~QueryPipeline() = default; QueryPipeline(QueryPipeline &&) = default; QueryPipeline(const QueryPipeline &) = delete; - QueryPipeline & operator= (QueryPipeline && rhs); + QueryPipeline & operator= (QueryPipeline && rhs) = default; QueryPipeline & operator= (const QueryPipeline & rhs) = delete; /// All pipes must have same header. @@ -73,7 +73,11 @@ public: /// Unite several pipelines together. Result pipeline would have common_header structure. /// If collector is used, it will collect only newly-added processors, but not processors from pipelines. - void unitePipelines(std::vector> pipelines, const Block & common_header, size_t max_threads_limit = 0); + static QueryPipeline unitePipelines( + std::vector> pipelines, + const Block & common_header, + size_t max_threads_limit = 0, + Processors * collected_processors = nullptr); PipelineExecutorPtr execute(); @@ -113,10 +117,9 @@ public: } /// Convert query pipeline to pipe. - Pipe getPipe() &&; + static Pipe getPipe(QueryPipeline pipeline) { return std::move(pipeline.pipe); } private: - /// Destruction order: processors, header, locks, temporary storages, local contexts Pipe pipe; IOutputFormat * output_format = nullptr; @@ -132,6 +135,8 @@ private: void initRowsBeforeLimit(); + void setCollectedProcessors(Processors * processors); + friend class QueryPipelineProcessorsCollector; }; From 5e21a590713e1e950c32c898747a746cb4fe5854 Mon Sep 17 00:00:00 2001 From: Jiading Guo Date: Wed, 5 Aug 2020 18:09:20 +0800 Subject: [PATCH 123/374] Update architecture.md --- docs/en/development/architecture.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/architecture.md b/docs/en/development/architecture.md index f4b74c19f4e..da5b4470704 100644 --- a/docs/en/development/architecture.md +++ b/docs/en/development/architecture.md @@ -120,7 +120,7 @@ There are ordinary functions and aggregate functions. For aggregate functions, s Ordinary functions don’t change the number of rows – they work as if they are processing each row independently. In fact, functions are not called for individual rows, but for `Block`’s of data to implement vectorized query execution. -There are some miscellaneous functions, like [blockSize](../sql-reference/functions/other-functions.md#function-blocksize), [rowNumberInBlock](../sql-reference/functions/other-functions.md#function-rownumberinblock), and \[runningAccumulate\](../sql-reference/functions/other-functions.md\#runningaccumulatexploit block processing and violate the independence of rows. +There are some miscellaneous functions, like [blockSize](../sql-reference/functions/other-functions.md#function-blocksize), [rowNumberInBlock](../sql-reference/functions/other-functions.md#function-rownumberinblock), and [runningAccumulate](../sql-reference/functions/other-functions.md#runningaccumulate), that exploit block processing and violate the independence of rows. ClickHouse has strong typing, so there’s no implicit type conversion. If a function doesn’t support a specific combination of types, it throws an exception. But functions can work (be overloaded) for many different combinations of types. For example, the `plus` function (to implement the `+` operator) works for any combination of numeric types: `UInt8` + `Float32`, `UInt16` + `Int8`, and so on. Also, some variadic functions can accept any number of arguments, such as the `concat` function. From 78568520107bfb5cacd9a1fbcdfa72ffa413c9b6 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 5 Aug 2020 19:36:10 +0300 Subject: [PATCH 124/374] Move parts to DiskS3 in parallel. --- src/Disks/DiskAsyncSupport.h | 15 +++++++ src/Disks/IDisk.cpp | 74 ++++++++++++++++++++++++++++++--- src/Disks/S3/DiskS3.cpp | 27 ++++++++++++ src/Disks/S3/DiskS3.h | 9 +++- src/Disks/S3/registerDiskS3.cpp | 1 + src/Interpreters/Context.cpp | 2 +- src/Interpreters/Context.h | 2 +- 7 files changed, 121 insertions(+), 9 deletions(-) create mode 100644 src/Disks/DiskAsyncSupport.h diff --git a/src/Disks/DiskAsyncSupport.h b/src/Disks/DiskAsyncSupport.h new file mode 100644 index 00000000000..14be095843e --- /dev/null +++ b/src/Disks/DiskAsyncSupport.h @@ -0,0 +1,15 @@ +#pragma once + +#include "future" + +namespace DB +{ + +class DiskAsyncSupport +{ +public: + virtual ~DiskAsyncSupport() = default; + virtual std::future runAsync(std::function task) = 0; +}; + +} diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 9d7424d1286..b4ad9c4b643 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -4,6 +4,8 @@ #include #include #include +#include "future" +#include "Disks/DiskAsyncSupport.h" namespace DB { @@ -18,6 +20,28 @@ bool IDisk::isDirectoryEmpty(const String & path) return !iterateDirectory(path)->isValid(); } +/// Executes task synchronously in case when destination disk doesn't support async operations. +class NoAsync : public DiskAsyncSupport +{ +public: + NoAsync() = default; + ~NoAsync() override = default; + std::future runAsync(std::function task) override + { + std::promise promise; + try + { + task(); + promise.set_value(); + } + catch (...) + { + promise.set_exception(std::current_exception()); + } + return promise.get_future(); + } +}; + void copyFile(IDisk & from_disk, const String & from_path, IDisk & to_disk, const String & to_path) { LOG_DEBUG(&Poco::Logger::get("IDisk"), "Copying from {} {} to {} {}.", from_disk.getName(), from_path, to_disk.getName(), to_path); @@ -27,26 +51,64 @@ void copyFile(IDisk & from_disk, const String & from_path, IDisk & to_disk, cons copyData(*in, *out); } -void IDisk::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) +std::future asyncCopy(IDisk & from_disk, const String & from_path, IDisk & to_disk, const String & to_path, DiskAsyncSupport & async) { - if (isFile(from_path)) + if (from_disk.isFile(from_path)) { - DB::copyFile(*this, from_path, *to_disk, to_path + fileName(from_path)); + return async.runAsync( + [&from_disk, &from_path, &to_disk, &to_path]() + { + DB::copyFile(from_disk, from_path, to_disk, to_path + fileName(from_path)); + } + ); } else { Poco::Path path(from_path); const String & dir_name = path.directory(path.depth() - 1); const String dest = to_path + dir_name + "/"; - to_disk->createDirectories(dest); + to_disk.createDirectories(dest); - for (auto it = iterateDirectory(from_path); it->isValid(); it->next()) + std::vector> futures; + std::promise promise; + + for (auto it = from_disk.iterateDirectory(from_path); it->isValid(); it->next()) + futures.push_back(asyncCopy(from_disk, it->path(), to_disk, dest, async)); + + for (auto & future : futures) { - copy(it->path(), to_disk, dest); + future.wait(); + try + { + future.get(); + } + catch (...) + { + promise.set_exception(std::current_exception()); + } } + + return promise.get_future(); } } +void IDisk::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) +{ + std::future future; + if (auto async = std::dynamic_pointer_cast(to_disk)) + { + future = asyncCopy(*this, from_path, *to_disk, to_path, *async); + } + else + { + auto no_async = std::make_unique(); + future = asyncCopy(*this, from_path, *to_disk, to_path, *no_async); + } + + future.wait(); + future.get(); +} + void IDisk::truncateFile(const String &, size_t) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate operation is not implemented for disk of type {}", getType()); diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 84178d81fab..bfa1d881b8e 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -420,6 +421,7 @@ DiskS3::DiskS3( String name_, std::shared_ptr client_, std::shared_ptr proxy_configuration_, + BackgroundProcessingPool & pool_, String bucket_, String s3_root_path_, String metadata_path_, @@ -429,6 +431,7 @@ DiskS3::DiskS3( : name(std::move(name_)) , client(std::move(client_)) , proxy_configuration(std::move(proxy_configuration_)) + , pool(pool_) , bucket(std::move(bucket_)) , s3_root_path(std::move(s3_root_path_)) , metadata_path(std::move(metadata_path_)) @@ -688,6 +691,30 @@ void DiskS3::setReadOnly(const String & path) Poco::File(metadata_path + path).setReadOnly(true); } +std::future DiskS3::runAsync(std::function task) +{ + std::promise promise; + + pool.addTask( + [&promise, &task]() + { + try + { + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Executing async task..."); + + task(); + promise.set_value(); + } + catch (...) + { + promise.set_exception(std::current_exception()); + } + return BackgroundProcessingPoolTaskResult::SUCCESS; + }); + + return promise.get_future(); +} + DiskS3Reservation::~DiskS3Reservation() { try diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 81ef3fc37d3..053cc5e0e38 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -1,6 +1,7 @@ #pragma once #include "Disks/DiskFactory.h" +#include "Disks/DiskAsyncSupport.h" #include "ProxyConfiguration.h" #include @@ -9,12 +10,14 @@ namespace DB { +class BackgroundProcessingPool; + /** * Storage for persisting data in S3 and metadata on the local disk. * Files are represented by file in local filesystem (clickhouse_root/disks/disk_name/path/to/file) * that contains S3 object key with actual data. */ -class DiskS3 : public IDisk +class DiskS3 : public IDisk, public DiskAsyncSupport { public: friend class DiskS3Reservation; @@ -23,6 +26,7 @@ public: String name_, std::shared_ptr client_, std::shared_ptr proxy_configuration_, + BackgroundProcessingPool & pool, String bucket_, String s3_root_path_, String metadata_path_, @@ -100,6 +104,8 @@ public: const String getType() const override { return "s3"; } + std::future runAsync(std::function task) override; + private: bool tryReserve(UInt64 bytes); @@ -107,6 +113,7 @@ private: const String name; std::shared_ptr client; std::shared_ptr proxy_configuration; + BackgroundProcessingPool & pool; const String bucket; const String s3_root_path; const String metadata_path; diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 34c28a62477..272a6e24b97 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -136,6 +136,7 @@ void registerDiskS3(DiskFactory & factory) name, client, proxy_config, + context.getBackgroundMovePool(), uri.bucket, uri.key, metadata_path, diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 01097a09cbb..4d7f8b93e1f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1362,7 +1362,7 @@ BackgroundProcessingPool & Context::getBackgroundPool() return *shared->background_pool; } -BackgroundProcessingPool & Context::getBackgroundMovePool() +BackgroundProcessingPool & Context::getBackgroundMovePool() const { auto lock = getLock(); if (!shared->background_move_pool) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index dfe2d00873f..830c5fc6bdb 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -499,7 +499,7 @@ public: BackgroundSchedulePool & getBufferFlushSchedulePool(); BackgroundProcessingPool & getBackgroundPool(); - BackgroundProcessingPool & getBackgroundMovePool(); + BackgroundProcessingPool & getBackgroundMovePool() const; BackgroundSchedulePool & getSchedulePool(); BackgroundSchedulePool & getDistributedSchedulePool(); From a3bc18bed2090c23686de55ad16f648ab2cfb8cc Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 6 Aug 2020 12:03:27 +0800 Subject: [PATCH 125/374] ISSUES-4006 try resolve review comment --- .../MySQL/MaterializeMySQLSettings.cpp | 2 +- .../MySQL/MaterializeMySQLSettings.h | 10 +-- .../MySQL/MaterializeMySQLSyncThread.cpp | 10 ++- src/Interpreters/Context.cpp | 6 -- src/Interpreters/Context.h | 1 - .../InterpreterExternalDDLQuery.cpp | 2 +- .../MySQL/InterpretersMySQLDDLQuery.cpp | 69 ++++++++++++------- .../MySQL/InterpretersMySQLDDLQuery.h | 18 ++--- src/Parsers/ASTExternalDDLQuery.h | 2 +- src/Parsers/MySQL/ASTAlterCommand.h | 10 +++ src/Parsers/MySQL/ASTAlterQuery.cpp | 25 ------- src/Parsers/MySQL/ASTAlterQuery.h | 12 +++- src/Parsers/MySQL/ASTCreateDefines.h | 12 +++- src/Parsers/MySQL/ASTCreateQuery.h | 11 +++ src/Parsers/MySQL/ASTDeclareColumn.h | 11 +++ src/Parsers/MySQL/ASTDeclareConstraint.h | 11 +++ src/Parsers/MySQL/ASTDeclareIndex.cpp | 2 - src/Parsers/MySQL/ASTDeclareIndex.h | 11 +++ src/Parsers/MySQL/ASTDeclareOption.cpp | 3 - src/Parsers/MySQL/ASTDeclareOption.h | 11 +++ src/Parsers/MySQL/ASTDeclarePartition.h | 11 +++ .../MySQL/ASTDeclarePartitionOptions.h | 11 +++ src/Parsers/MySQL/ASTDeclareReference.h | 11 +++ src/Parsers/MySQL/ASTDeclareSubPartition.h | 11 +++ 24 files changed, 200 insertions(+), 83 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMySQLSettings.cpp b/src/Databases/MySQL/MaterializeMySQLSettings.cpp index 688de941692..609ce011f91 100644 --- a/src/Databases/MySQL/MaterializeMySQLSettings.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSettings.cpp @@ -25,7 +25,7 @@ void MaterializeMySQLSettings::loadFromQuery(ASTStorage & storage_def) catch (Exception & e) { if (e.code() == ErrorCodes::UNKNOWN_SETTING) - throw Exception(e.message() + " for storage " + storage_def.engine->name, ErrorCodes::BAD_ARGUMENTS); + throw Exception(e.message() + " for database " + storage_def.engine->name, ErrorCodes::BAD_ARGUMENTS); else e.rethrow(); } diff --git a/src/Databases/MySQL/MaterializeMySQLSettings.h b/src/Databases/MySQL/MaterializeMySQLSettings.h index effd231ed45..539471becfb 100644 --- a/src/Databases/MySQL/MaterializeMySQLSettings.h +++ b/src/Databases/MySQL/MaterializeMySQLSettings.h @@ -9,11 +9,11 @@ namespace DB class ASTStorage; #define LIST_OF_MATERIALIZE_MODE_SETTINGS(M) \ - M(UInt64, max_rows_in_buffer, DEFAULT_BLOCK_SIZE, "", 0) \ - M(UInt64, max_bytes_in_buffer, DBMS_DEFAULT_BUFFER_SIZE, "", 0) \ - M(UInt64, max_rows_in_buffers, DEFAULT_BLOCK_SIZE, "", 0) \ - M(UInt64, max_bytes_in_buffers, DBMS_DEFAULT_BUFFER_SIZE, "", 0) \ - M(UInt64, max_flush_data_time, 1000, "", 0) \ + M(UInt64, max_rows_in_buffer, DEFAULT_BLOCK_SIZE, "Max rows that data is allowed to cache in memory(for single table and the cache data unable to query). when rows is exceeded, the data will be materialized", 0) \ + M(UInt64, max_bytes_in_buffer, DBMS_DEFAULT_BUFFER_SIZE, "Max bytes that data is allowed to cache in memory(for single table and the cache data unable to query). when rows is exceeded, the data will be materialized", 0) \ + M(UInt64, max_rows_in_buffers, DEFAULT_BLOCK_SIZE, "Max rows that data is allowed to cache in memory(for database and the cache data unable to query). when rows is exceeded, the data will be materialized", 0) \ + M(UInt64, max_bytes_in_buffers, DBMS_DEFAULT_BUFFER_SIZE, "Max bytes that data is allowed to cache in memory(for database and the cache data unable to query). when rows is exceeded, the data will be materialized", 0) \ + M(UInt64, max_flush_data_time, 1000, "Max milliseconds that data is allowed to cache in memory(for database and the cache data unable to query). when this time is exceeded, the data will be materialized", 0) \ M(UInt64, max_wait_time_when_mysql_unavailable, 1000, "", 0) \ DECLARE_SETTINGS_TRAITS(MaterializeMySQLSettingsTraits, LIST_OF_MATERIALIZE_MODE_SETTINGS) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 4ec6bc320b5..43d1295e5c4 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -43,7 +43,10 @@ static BlockIO tryToExecuteQuery(const String & query_to_execute, const Context { Context context(context_); CurrentThread::QueryScope query_scope(context); - context.unsafeSetCurrentDatabase(database); + + if (!database.empty()) + context.setCurrentDatabase(database); + context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; context.setCurrentQueryId(""); // generate random query_id @@ -250,7 +253,7 @@ static inline void dumpDataForTables( { const auto & table_name = iterator->first; String comment = "Materialize MySQL step 1: execute MySQL DDL for dump data"; - tryToExecuteQuery(query_prefix + " " + iterator->second, context, mysql_database_name, comment); /// create table. + tryToExecuteQuery(query_prefix + " " + iterator->second, context, database_name, comment); /// create table. auto out = std::make_shared(master_info.version, getTableOutput(database_name, table_name, context)); MySQLBlockInputStream input( @@ -579,7 +582,8 @@ void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr try { String comment = "Materialize MySQL step 2: execute MySQL DDL for sync data"; - tryToExecuteQuery(query_prefix + query_event.query, global_context, query_event.schema, comment); + String event_database = query_event.schema == mysql_database_name ? database_name : ""; + tryToExecuteQuery(query_prefix + query_event.query, global_context, event_database, comment); } catch (Exception & exception) { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d547cb4bb3b..14fcf05356d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1070,12 +1070,6 @@ void Context::setCurrentDatabase(const String & name) calculateAccessRights(); } -void Context::unsafeSetCurrentDatabase(const String & name) -{ - auto lock = getLock(); - current_database = name; -} - void Context::setCurrentQueryId(const String & query_id) { if (!client_info.current_query_id.empty()) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 1bc3db46652..dfe2d00873f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -353,7 +353,6 @@ public: void setCurrentDatabase(const String & name); void setCurrentQueryId(const String & query_id); - void unsafeSetCurrentDatabase(const String & name); void killCurrentQuery(); diff --git a/src/Interpreters/InterpreterExternalDDLQuery.cpp b/src/Interpreters/InterpreterExternalDDLQuery.cpp index 8f1c9da22b7..89b8e598ac9 100644 --- a/src/Interpreters/InterpreterExternalDDLQuery.cpp +++ b/src/Interpreters/InterpreterExternalDDLQuery.cpp @@ -46,7 +46,6 @@ BlockIO InterpreterExternalDDLQuery::execute() if (arguments.size() != 2 || !arguments[0]->as() || !arguments[1]->as()) throw Exception("MySQL External require two identifier arguments.", ErrorCodes::BAD_ARGUMENTS); -#endif if (external_ddl_query.external_ddl->as()) return MySQLInterpreter::InterpreterMySQLDropQuery( external_ddl_query.external_ddl, context, getIdentifierName(arguments[0]), @@ -63,6 +62,7 @@ BlockIO InterpreterExternalDDLQuery::execute() return MySQLInterpreter::InterpreterMySQLCreateQuery( external_ddl_query.external_ddl, context, getIdentifierName(arguments[0]), getIdentifierName(arguments[1])).execute(); +#endif } return BlockIO(); diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 682327699e3..ca39a8ae4fc 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -36,6 +36,33 @@ namespace ErrorCodes namespace MySQLInterpreter { +static inline String resolveDatabase( + const String & database_in_query, const String & replica_mysql_database, const String & replica_clickhouse_database, const Context & context) +{ + if (!database_in_query.empty()) + { + if (database_in_query == replica_mysql_database) + { + /// USE other_database_name; CREATE TABLE replica_mysql_database.table_name; + /// USE replica_mysql_database; CREATE TABLE replica_mysql_database.table_name; + return replica_clickhouse_database; + } + else + { + /// USE other_database_name; CREATE TABLE other_database_name.table_name; + /// USE replica_mysql_database; CREATE TABLE other_database_name.table_name; + return ""; + } + } + + /// When USE other_database_name; CREATE TABLE table_name; + /// context.getCurrentDatabase() is always return `default database` + /// When USE replica_mysql_database; CREATE TABLE table_name; + /// context.getCurrentDatabase() is always return replica_clickhouse_database + const String & current_database = context.getCurrentDatabase(); + return current_database != replica_clickhouse_database ? "" : replica_clickhouse_database; +} + static inline NamesAndTypesList getColumnsList(ASTExpressionList * columns_define) { NamesAndTypesList columns_name_and_type; @@ -295,12 +322,10 @@ void InterpreterCreateImpl::validate(const InterpreterCreateImpl::TQuery & creat } ASTPtr InterpreterCreateImpl::getRewrittenQuery( - const TQuery & create_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db) + const TQuery & create_query, const Context & context, const String & mapped_to_database, const String & mysql_database) { auto rewritten_query = std::make_shared(); - const auto & database_name = context.resolveDatabase(create_query.database); - - if (database_name != filter_mysql_db) + if (resolveDatabase(create_query.database, mysql_database, mapped_to_database, context) != mapped_to_database) return {}; const auto & create_defines = create_query.columns_list->as(); @@ -309,7 +334,7 @@ ASTPtr InterpreterCreateImpl::getRewrittenQuery( const auto & [primary_keys, unique_keys, keys, increment_columns] = getKeys(create_defines->columns, create_defines->indices, context, columns_name_and_type); if (primary_keys.empty()) - throw Exception("The " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(create_query.table) + throw Exception("The " + backQuoteIfNeed(mysql_database) + "." + backQuoteIfNeed(create_query.table) + " cannot be materialized, because there is no primary keys.", ErrorCodes::NOT_IMPLEMENTED); auto columns = std::make_shared(); @@ -333,7 +358,7 @@ ASTPtr InterpreterCreateImpl::getRewrittenQuery( storage->set(storage->engine, makeASTFunction("ReplacingMergeTree", std::make_shared(version_column_name))); - rewritten_query->database = clickhouse_db; + rewritten_query->database = mapped_to_database; rewritten_query->table = create_query.table; rewritten_query->if_not_exists = create_query.if_not_exists; rewritten_query->set(rewritten_query->storage, storage); @@ -347,16 +372,16 @@ void InterpreterDropImpl::validate(const InterpreterDropImpl::TQuery & /*query*/ } ASTPtr InterpreterDropImpl::getRewrittenQuery( - const InterpreterDropImpl::TQuery & drop_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db) + const InterpreterDropImpl::TQuery & drop_query, const Context & context, const String & mapped_to_database, const String & mysql_database) { - const auto & database_name = context.resolveDatabase(drop_query.database); + const auto & database_name = resolveDatabase(drop_query.database, mysql_database, mapped_to_database, context); /// Skip drop database|view|dictionary - if (database_name != filter_mysql_db || drop_query.table.empty() || drop_query.is_view || drop_query.is_dictionary) + if (database_name != mapped_to_database || drop_query.table.empty() || drop_query.is_view || drop_query.is_dictionary) return {}; ASTPtr rewritten_query = drop_query.clone(); - rewritten_query->as()->database = clickhouse_db; + rewritten_query->as()->database = mapped_to_database; return rewritten_query; } @@ -367,23 +392,23 @@ void InterpreterRenameImpl::validate(const InterpreterRenameImpl::TQuery & renam } ASTPtr InterpreterRenameImpl::getRewrittenQuery( - const InterpreterRenameImpl::TQuery & rename_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db) + const InterpreterRenameImpl::TQuery & rename_query, const Context & context, const String & mapped_to_database, const String & mysql_database) { ASTRenameQuery::Elements elements; for (const auto & rename_element : rename_query.elements) { - const auto & to_database = context.resolveDatabase(rename_element.to.database); - const auto & from_database = context.resolveDatabase(rename_element.from.database); + const auto & to_database = resolveDatabase(rename_element.to.database, mysql_database, mapped_to_database, context); + const auto & from_database = resolveDatabase(rename_element.from.database, mysql_database, mapped_to_database, context); - if (to_database != from_database) + if ((from_database == mapped_to_database || to_database == mapped_to_database) && to_database != from_database) throw Exception("Cannot rename with other database for external ddl query.", ErrorCodes::NOT_IMPLEMENTED); - if (from_database == filter_mysql_db) + if (from_database == mapped_to_database) { elements.push_back(ASTRenameQuery::Element()); - elements.back().from.database = clickhouse_db; + elements.back().from.database = mapped_to_database; elements.back().from.table = rename_element.from.table; - elements.back().to.database = clickhouse_db; + elements.back().to.database = mapped_to_database; elements.back().to.table = rename_element.to.table; } } @@ -401,15 +426,13 @@ void InterpreterAlterImpl::validate(const InterpreterAlterImpl::TQuery & /*query } ASTPtr InterpreterAlterImpl::getRewrittenQuery( - const InterpreterAlterImpl::TQuery & alter_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db) + const InterpreterAlterImpl::TQuery & alter_query, const Context & context, const String & mapped_to_database, const String & mysql_database) { - const auto & database_name = context.resolveDatabase(alter_query.database); - - if (database_name != filter_mysql_db) + if (resolveDatabase(alter_query.database, mysql_database, mapped_to_database, context) != mapped_to_database) return {}; auto rewritten_query = std::make_shared(); - rewritten_query->database = clickhouse_db; + rewritten_query->database = mapped_to_database; rewritten_query->table = alter_query.table; rewritten_query->set(rewritten_query->command_list, std::make_shared()); @@ -448,7 +471,7 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( if (default_after_column.empty()) { - StoragePtr storage = DatabaseCatalog::instance().getTable({clickhouse_db, alter_query.table}, context); + StoragePtr storage = DatabaseCatalog::instance().getTable({mapped_to_database, alter_query.table}, context); Block storage_header = storage->getInMemoryMetadataPtr()->getSampleBlock(); /// Put the sign and version columns last diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h index d8920194ea0..8fc01c78d9a 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h @@ -21,7 +21,7 @@ struct InterpreterDropImpl static void validate(const TQuery & query, const Context & context); - static ASTPtr getRewrittenQuery(const TQuery & drop_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db); + static ASTPtr getRewrittenQuery(const TQuery & drop_query, const Context & context, const String & mapped_to_database, const String & mysql_database); }; struct InterpreterAlterImpl @@ -30,7 +30,7 @@ struct InterpreterAlterImpl static void validate(const TQuery & query, const Context & context); - static ASTPtr getRewrittenQuery(const TQuery & alter_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db); + static ASTPtr getRewrittenQuery(const TQuery & alter_query, const Context & context, const String & mapped_to_database, const String & mysql_database); }; struct InterpreterRenameImpl @@ -39,7 +39,7 @@ struct InterpreterRenameImpl static void validate(const TQuery & query, const Context & context); - static ASTPtr getRewrittenQuery(const TQuery & rename_query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db); + static ASTPtr getRewrittenQuery(const TQuery & rename_query, const Context & context, const String & mapped_to_database, const String & mysql_database); }; struct InterpreterCreateImpl @@ -48,15 +48,15 @@ struct InterpreterCreateImpl static void validate(const TQuery & query, const Context & context); - static ASTPtr getRewrittenQuery(const TQuery & query, const Context & context, const String & clickhouse_db, const String & filter_mysql_db); + static ASTPtr getRewrittenQuery(const TQuery & create_query, const Context & context, const String & mapped_to_database, const String & mysql_database); }; template class InterpreterMySQLDDLQuery : public IInterpreter { public: - InterpreterMySQLDDLQuery(const ASTPtr & query_ptr_, Context & context_, const String & clickhouse_db_, const String & mysql_db_) - : query_ptr(query_ptr_), context(context_), clickhouse_db(clickhouse_db_), mysql_db(mysql_db_) + InterpreterMySQLDDLQuery(const ASTPtr & query_ptr_, Context & context_, const String & mapped_to_database_, const String & mysql_database_) + : query_ptr(query_ptr_), context(context_), mapped_to_database(mapped_to_database_), mysql_database(mysql_database_) { } @@ -65,7 +65,7 @@ public: const typename InterpreterImpl::TQuery & query = query_ptr->as(); InterpreterImpl::validate(query, context); - ASTPtr rewritten_query = InterpreterImpl::getRewrittenQuery(query, context, clickhouse_db, mysql_db); + ASTPtr rewritten_query = InterpreterImpl::getRewrittenQuery(query, context, mapped_to_database, mysql_database); if (rewritten_query) return executeQuery("/* Rewritten MySQL DDL Query */ " + queryToString(rewritten_query), context, true); @@ -76,8 +76,8 @@ public: private: ASTPtr query_ptr; Context & context; - const String clickhouse_db; - const String mysql_db; + const String mapped_to_database; + const String mysql_database; }; using InterpreterMySQLDropQuery = InterpreterMySQLDDLQuery; diff --git a/src/Parsers/ASTExternalDDLQuery.h b/src/Parsers/ASTExternalDDLQuery.h index d7b3bdd932f..d659c5c0bb8 100644 --- a/src/Parsers/ASTExternalDDLQuery.h +++ b/src/Parsers/ASTExternalDDLQuery.h @@ -33,7 +33,7 @@ public: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked stacked) const override { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXTERNAL DDL FROM "; + settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXTERNAL DDL FROM " << (settings.hilite ? hilite_none : ""); from->formatImpl(settings, state, stacked); external_ddl->formatImpl(settings, state, stacked); } diff --git a/src/Parsers/MySQL/ASTAlterCommand.h b/src/Parsers/MySQL/ASTAlterCommand.h index 3667116952b..fa8fee83fd3 100644 --- a/src/Parsers/MySQL/ASTAlterCommand.h +++ b/src/Parsers/MySQL/ASTAlterCommand.h @@ -8,6 +8,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} namespace MySQLParser { @@ -68,6 +72,12 @@ public: ASTPtr clone() const override; String getID(char delim) const override { return "AlterCommand" + (delim + std::to_string(static_cast(type))); } + +protected: + void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const override + { + throw Exception("Method formatImpl is not supported by MySQLParser::ASTAlterCommand.", ErrorCodes::NOT_IMPLEMENTED); + } }; class ParserAlterCommand : public IParserBase diff --git a/src/Parsers/MySQL/ASTAlterQuery.cpp b/src/Parsers/MySQL/ASTAlterQuery.cpp index da2dc178373..92814e42d82 100644 --- a/src/Parsers/MySQL/ASTAlterQuery.cpp +++ b/src/Parsers/MySQL/ASTAlterQuery.cpp @@ -28,31 +28,6 @@ ASTPtr ASTAlterQuery::clone() const return res; } -/*void ASTAlterQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState & state, IAST::FormatStateStacked frame) const -{ - frame.need_parens = false; - - std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' '); - - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (settings.hilite ? hilite_none : ""); - - if (!table.empty()) - { - if (!database.empty()) - { - settings.ostr << indent_str << backQuoteIfNeed(database); - settings.ostr << "."; - } - settings.ostr << indent_str << backQuoteIfNeed(table); - } - - settings.ostr << settings.nl_or_ws; - FormatStateStacked frame_nested = frame; - frame_nested.need_parens = false; - ++frame_nested.indent; -// static_cast(command_list)->formatImpl(settings, state, frame_nested); -}*/ - bool ParserAlterQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr table; diff --git a/src/Parsers/MySQL/ASTAlterQuery.h b/src/Parsers/MySQL/ASTAlterQuery.h index 37f5687a456..a6987acb327 100644 --- a/src/Parsers/MySQL/ASTAlterQuery.h +++ b/src/Parsers/MySQL/ASTAlterQuery.h @@ -8,6 +8,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + namespace MySQLParser { @@ -22,8 +27,11 @@ public: String getID(char delim) const override { return "AlterQuery" + (delim + database) + delim + table; } -/*protected: - void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;*/ +protected: + void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const override + { + throw Exception("Method formatImpl is not supported by MySQLParser::ASTAlterQuery.", ErrorCodes::NOT_IMPLEMENTED); + } }; class ParserAlterQuery : public IParserBase diff --git a/src/Parsers/MySQL/ASTCreateDefines.h b/src/Parsers/MySQL/ASTCreateDefines.h index 72adb1c745a..95fb7716f3a 100644 --- a/src/Parsers/MySQL/ASTCreateDefines.h +++ b/src/Parsers/MySQL/ASTCreateDefines.h @@ -7,10 +7,14 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + namespace MySQLParser { - class ASTCreateDefines : public IAST { public: @@ -21,6 +25,12 @@ public: ASTPtr clone() const override; String getID(char) const override { return "Create definitions"; } + +protected: + void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const override + { + throw Exception("Method formatImpl is not supported by MySQLParser::ASTCreateDefines.", ErrorCodes::NOT_IMPLEMENTED); + } }; class ParserCreateDefines : public IParserBase diff --git a/src/Parsers/MySQL/ASTCreateQuery.h b/src/Parsers/MySQL/ASTCreateQuery.h index 2d1deb23d4b..ceacdd2cd41 100644 --- a/src/Parsers/MySQL/ASTCreateQuery.h +++ b/src/Parsers/MySQL/ASTCreateQuery.h @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + namespace MySQLParser { @@ -25,6 +30,12 @@ public: ASTPtr clone() const override; String getID(char) const override { return "create query"; } + +protected: + void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const override + { + throw Exception("Method formatImpl is not supported by MySQLParser::ASTCreateQuery.", ErrorCodes::NOT_IMPLEMENTED); + } }; class ParserCreateQuery : public IParserBase diff --git a/src/Parsers/MySQL/ASTDeclareColumn.h b/src/Parsers/MySQL/ASTDeclareColumn.h index 75683c2ff4c..6f9f50225a2 100644 --- a/src/Parsers/MySQL/ASTDeclareColumn.h +++ b/src/Parsers/MySQL/ASTDeclareColumn.h @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + namespace MySQLParser { @@ -19,6 +24,12 @@ public: ASTPtr clone() const override; String getID(char /*delimiter*/) const override { return "Column definition"; } + +protected: + void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const override + { + throw Exception("Method formatImpl is not supported by MySQLParser::ASTDeclareColumn.", ErrorCodes::NOT_IMPLEMENTED); + } }; class ParserDeclareColumn : public IParserBase diff --git a/src/Parsers/MySQL/ASTDeclareConstraint.h b/src/Parsers/MySQL/ASTDeclareConstraint.h index 6af40e499eb..8b0153bcd53 100644 --- a/src/Parsers/MySQL/ASTDeclareConstraint.h +++ b/src/Parsers/MySQL/ASTDeclareConstraint.h @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + namespace MySQLParser { @@ -19,6 +24,12 @@ public: ASTPtr clone() const override; String getID(char /*delimiter*/) const override { return "constraint declaration"; } + +protected: + void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const override + { + throw Exception("Method formatImpl is not supported by MySQLParser::ASTDeclareConstraint.", ErrorCodes::NOT_IMPLEMENTED); + } }; class ParserDeclareConstraint : public IParserBase diff --git a/src/Parsers/MySQL/ASTDeclareIndex.cpp b/src/Parsers/MySQL/ASTDeclareIndex.cpp index 0a6a2095261..8e6e9d43793 100644 --- a/src/Parsers/MySQL/ASTDeclareIndex.cpp +++ b/src/Parsers/MySQL/ASTDeclareIndex.cpp @@ -64,7 +64,6 @@ static inline bool parseDeclareOrdinaryIndex(IParser::Pos & pos, String & index_ ParserKeyword k_key("KEY"); ParserKeyword k_index("INDEX"); - ParserExpression p_expression; ParserIdentifier p_identifier; if (ParserKeyword("SPATIAL").ignore(pos, expected)) @@ -172,7 +171,6 @@ bool ParserDeclareIndex::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & ASTPtr index_columns; ASTPtr index_options; ASTPtr declare_reference; - ParserIndexColumn p_expression; ParserDeclareOptions p_index_options{ { diff --git a/src/Parsers/MySQL/ASTDeclareIndex.h b/src/Parsers/MySQL/ASTDeclareIndex.h index 8488b5d2144..faa8f1378dd 100644 --- a/src/Parsers/MySQL/ASTDeclareIndex.h +++ b/src/Parsers/MySQL/ASTDeclareIndex.h @@ -8,6 +8,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + namespace MySQLParser { @@ -23,6 +28,12 @@ public: ASTPtr clone() const override; String getID(char /*delimiter*/) const override { return "index declaration"; } + +protected: + void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const override + { + throw Exception("Method formatImpl is not supported by MySQLParser::ASTDeclareIndex.", ErrorCodes::NOT_IMPLEMENTED); + } }; class ParserDeclareIndex : public IParserBase diff --git a/src/Parsers/MySQL/ASTDeclareOption.cpp b/src/Parsers/MySQL/ASTDeclareOption.cpp index e8adea8a8d2..92ac5f0343e 100644 --- a/src/Parsers/MySQL/ASTDeclareOption.cpp +++ b/src/Parsers/MySQL/ASTDeclareOption.cpp @@ -49,9 +49,6 @@ bool ParserDeclareOptionImpl::parseImpl(Pos & pos, ASTPtr & node, Exp if (!option_describe.value_parser->parse(pos, value, expected)) return false; - /*const auto & changes_iterator = changes.find(option_describe.option_name); - if (changes_iterator != changes.end()) - throw Exception("Duplicate options declare", ErrorCodes::)*/ found = true; changes.insert(std::make_pair(option_describe.option_name, value)); break; diff --git a/src/Parsers/MySQL/ASTDeclareOption.h b/src/Parsers/MySQL/ASTDeclareOption.h index fa22698175a..24800371061 100644 --- a/src/Parsers/MySQL/ASTDeclareOption.h +++ b/src/Parsers/MySQL/ASTDeclareOption.h @@ -7,6 +7,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + namespace MySQLParser { @@ -30,6 +35,12 @@ public: ASTPtr clone() const override; String getID(char /*delimiter*/) const override { return "options declaration"; } + +protected: + void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const override + { + throw Exception("Method formatImpl is not supported by MySQLParser::ASTDeclareOptions.", ErrorCodes::NOT_IMPLEMENTED); + } }; class ParserAlwaysTrue : public IParserBase diff --git a/src/Parsers/MySQL/ASTDeclarePartition.h b/src/Parsers/MySQL/ASTDeclarePartition.h index ef076dcd6cf..232fea57561 100644 --- a/src/Parsers/MySQL/ASTDeclarePartition.h +++ b/src/Parsers/MySQL/ASTDeclarePartition.h @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + namespace MySQLParser { @@ -21,6 +26,12 @@ public: ASTPtr clone() const override; String getID(char /*delimiter*/) const override { return "partition declaration"; } + +protected: + void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const override + { + throw Exception("Method formatImpl is not supported by MySQLParser::ASTDeclarePartition.", ErrorCodes::NOT_IMPLEMENTED); + } }; class ParserDeclarePartition : public IParserBase diff --git a/src/Parsers/MySQL/ASTDeclarePartitionOptions.h b/src/Parsers/MySQL/ASTDeclarePartitionOptions.h index 3585be9df61..9e29a5cbbae 100644 --- a/src/Parsers/MySQL/ASTDeclarePartitionOptions.h +++ b/src/Parsers/MySQL/ASTDeclarePartitionOptions.h @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + namespace MySQLParser { @@ -23,6 +28,12 @@ public: ASTPtr clone() const override; String getID(char /*delimiter*/) const override { return "partition options declaration"; } + +protected: + void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const override + { + throw Exception("Method formatImpl is not supported by MySQLParser::ASTDeclarePartitionOptions.", ErrorCodes::NOT_IMPLEMENTED); + } }; class ParserDeclarePartitionOptions : public IParserBase diff --git a/src/Parsers/MySQL/ASTDeclareReference.h b/src/Parsers/MySQL/ASTDeclareReference.h index 41cd6597dfb..c003bd7a16c 100644 --- a/src/Parsers/MySQL/ASTDeclareReference.h +++ b/src/Parsers/MySQL/ASTDeclareReference.h @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + namespace MySQLParser { @@ -37,6 +42,12 @@ public: ASTPtr clone() const override; String getID(char /*delimiter*/) const override { return "subpartition declaration"; } + +protected: + void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const override + { + throw Exception("Method formatImpl is not supported by MySQLParser::ASTDeclareReference.", ErrorCodes::NOT_IMPLEMENTED); + } }; class ParserDeclareReference : public IParserBase diff --git a/src/Parsers/MySQL/ASTDeclareSubPartition.h b/src/Parsers/MySQL/ASTDeclareSubPartition.h index 018cf2c8c4e..4f00a39c99a 100644 --- a/src/Parsers/MySQL/ASTDeclareSubPartition.h +++ b/src/Parsers/MySQL/ASTDeclareSubPartition.h @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + namespace MySQLParser { @@ -18,6 +23,12 @@ public: ASTPtr clone() const override; String getID(char /*delimiter*/) const override { return "subpartition declaration"; } + +protected: + void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const override + { + throw Exception("Method formatImpl is not supported by MySQLParser::ASTDeclareSubPartition.", ErrorCodes::NOT_IMPLEMENTED); + } }; class ParserDeclareSubPartition : public IParserBase From 20e63d22711c00fc7fa4587211fbb9b1f65fbc8b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 6 Aug 2020 15:24:05 +0300 Subject: [PATCH 126/374] Refactor Pipe [part 6] --- src/Client/Connection.cpp | 10 ++- src/Core/ExternalTable.cpp | 10 ++- src/DataStreams/BlockIO.cpp | 2 +- src/DataStreams/RemoteQueryExecutor.cpp | 15 +--- .../gtest_blocks_size_merging_streams.cpp | 28 +++--- .../ClusterProxy/SelectStreamFactory.cpp | 4 +- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Processors/Pipe.cpp | 90 ++++++++----------- src/Processors/Pipe.h | 32 ++++--- src/Processors/QueryPipeline.cpp | 8 +- src/Processors/QueryPipeline.h | 3 + .../QueryPlan/ReadFromStorageStep.cpp | 2 - src/Processors/QueryPlan/UnionStep.cpp | 2 +- src/Storages/LiveView/StorageBlocks.h | 4 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 11 +-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 4 +- src/Storages/RabbitMQ/StorageRabbitMQ.h | 2 +- src/Storages/StorageBuffer.cpp | 17 ++-- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageGenerateRandom.cpp | 2 +- src/Storages/StorageHDFS.cpp | 2 +- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMerge.cpp | 24 ++--- src/Storages/StorageMerge.h | 2 +- src/Storages/StorageStripeLog.cpp | 4 +- src/Storages/StorageView.cpp | 2 +- src/Storages/System/IStorageSystemOneBlock.h | 7 +- src/Storages/System/StorageSystemColumns.cpp | 8 +- src/Storages/System/StorageSystemColumns.h | 2 +- .../System/StorageSystemDetachedParts.cpp | 6 +- src/Storages/System/StorageSystemDisks.cpp | 7 +- src/Storages/System/StorageSystemDisks.h | 2 +- src/Storages/System/StorageSystemNumbers.cpp | 33 +++---- src/Storages/System/StorageSystemNumbers.h | 2 +- src/Storages/System/StorageSystemOne.cpp | 7 +- src/Storages/System/StorageSystemOne.h | 2 +- .../System/StorageSystemPartsBase.cpp | 7 +- src/Storages/System/StorageSystemPartsBase.h | 2 +- src/Storages/System/StorageSystemReplicas.cpp | 8 +- src/Storages/System/StorageSystemReplicas.h | 2 +- .../System/StorageSystemStoragePolicies.cpp | 7 +- .../System/StorageSystemStoragePolicies.h | 2 +- src/Storages/System/StorageSystemTables.cpp | 7 +- src/Storages/System/StorageSystemTables.h | 2 +- src/Storages/System/StorageSystemZeros.cpp | 7 +- src/Storages/System/StorageSystemZeros.h | 2 +- 50 files changed, 209 insertions(+), 211 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 02e107db9a6..e93e84c5e94 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include @@ -581,10 +582,13 @@ void Connection::sendExternalTablesData(ExternalTablesData & data) PipelineExecutorPtr executor; auto on_cancel = [& executor]() { executor->cancel(); }; - auto sink = std::make_shared(elem->pipe->getHeader(), *this, *elem, std::move(on_cancel)); - DB::connect(elem->pipe->getPort(), sink->getPort()); + if (elem->pipe->numOutputPorts() > 1) + elem->pipe->addTransform(std::make_shared(elem->pipe->getHeader(), elem->pipe->numOutputPorts())); - auto processors = std::move(*elem->pipe).detachProcessors(); + auto sink = std::make_shared(elem->pipe->getHeader(), *this, *elem, std::move(on_cancel)); + DB::connect(*elem->pipe->getOutputPort(0), sink->getPort()); + + auto processors = Pipe::detachProcessors(std::move(*elem->pipe)); processors.push_back(sink); executor = std::make_shared(processors); diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 3639a109b42..a6e1407c1f2 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -170,10 +171,13 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, BlockOutputStreamPtr output = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), context); /// Write data - auto sink = std::make_shared(std::move(output)); - connect(data->pipe->getPort(), sink->getPort()); + if (data->pipe->numOutputPorts() > 1) + data->pipe->addTransform(std::make_shared(data->pipe->getHeader(), data->pipe->numOutputPorts())); - auto processors = std::move(*data->pipe).detachProcessors(); + auto sink = std::make_shared(std::move(output)); + connect(*data->pipe->getOutputPort(0), sink->getPort()); + + auto processors = Pipe::detachProcessors(std::move(*data->pipe)); processors.push_back(std::move(sink)); auto executor = std::make_shared(processors); diff --git a/src/DataStreams/BlockIO.cpp b/src/DataStreams/BlockIO.cpp index 150995962bf..3bbef4471db 100644 --- a/src/DataStreams/BlockIO.cpp +++ b/src/DataStreams/BlockIO.cpp @@ -42,7 +42,7 @@ void BlockIO::reset() in.reset(); if (process_list_entry) process_list_entry->get().releaseQueryStreams(); - pipeline = QueryPipeline(); + pipeline.reset(); process_list_entry.reset(); /// TODO Do we need also reset callbacks? In which order? diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index be09cd94814..5de396ec967 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -2,10 +2,8 @@ #include #include -#include #include #include -#include #include #include #include @@ -322,9 +320,7 @@ void RemoteQueryExecutor::sendExternalTables() auto metadata_snapshot = cur->getInMemoryMetadataPtr(); QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage(context); - Pipes pipes; - - pipes = cur->read( + Pipe pipe = cur->read( metadata_snapshot->getColumns().getNamesOfPhysical(), metadata_snapshot, {}, context, read_from_table_stage, DEFAULT_BLOCK_SIZE, 1); @@ -332,16 +328,11 @@ void RemoteQueryExecutor::sendExternalTables() auto data = std::make_unique(); data->table_name = table.first; - if (pipes.empty()) + if (pipe.empty()) data->pipe = std::make_unique( std::make_shared(metadata_snapshot->getSampleBlock(), Chunk())); - else if (pipes.size() == 1) - data->pipe = std::make_unique(std::move(pipes.front())); else - { - auto concat = std::make_shared(pipes.front().getHeader(), pipes.size()); - data->pipe = std::make_unique(std::move(pipes), std::move(concat)); - } + data->pipe = std::make_unique(std::move(pipe)); res.emplace_back(std::move(data)); } diff --git a/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp b/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp index c0d1a4979e1..0ce450c4e6c 100644 --- a/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp @@ -31,7 +31,7 @@ static Block getBlockWithSize(const std::vector & columns, size_t r } -static Pipes getInputStreams(const std::vector & column_names, const std::vector> & block_sizes) +static Pipe getInputStreams(const std::vector & column_names, const std::vector> & block_sizes) { Pipes pipes; for (auto [block_size_in_bytes, blocks_count, stride] : block_sizes) @@ -42,12 +42,12 @@ static Pipes getInputStreams(const std::vector & column_names, cons blocks.push_back(getBlockWithSize(column_names, block_size_in_bytes, stride, start)); pipes.emplace_back(std::make_shared(std::make_shared(std::move(blocks)))); } - return pipes; + return Pipe::unitePipes(std::move(pipes)); } -static Pipes getInputStreamsEqualStride(const std::vector & column_names, const std::vector> & block_sizes) +static Pipe getInputStreamsEqualStride(const std::vector & column_names, const std::vector> & block_sizes) { Pipes pipes; size_t i = 0; @@ -60,7 +60,7 @@ static Pipes getInputStreamsEqualStride(const std::vector & column_ pipes.emplace_back(std::make_shared(std::make_shared(std::move(blocks)))); i++; } - return pipes; + return Pipe::unitePipes(std::move(pipes)); } @@ -79,15 +79,17 @@ TEST(MergingSortedTest, SimpleBlockSizeTest) { std::vector key_columns{"K1", "K2", "K3"}; auto sort_description = getSortDescription(key_columns); - auto pipes = getInputStreams(key_columns, {{5, 1, 1}, {10, 1, 2}, {21, 1, 3}}); + auto pipe = getInputStreams(key_columns, {{5, 1, 1}, {10, 1, 2}, {21, 1, 3}}); - EXPECT_EQ(pipes.size(), 3); + EXPECT_EQ(pipe.numOutputPorts(), 3); - auto transform = std::make_shared(pipes.front().getHeader(), pipes.size(), sort_description, + auto transform = std::make_shared(pipe.getHeader(), pipe.numOutputPorts(), sort_description, DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true); + pipe.addTransform(std::move(transform)); + QueryPipeline pipeline; - pipeline.init(Pipe(std::move(pipes), std::move(transform))); + pipeline.init(std::move(pipe)); pipeline.setMaxThreads(1); auto stream = std::make_shared(std::move(pipeline)); @@ -122,15 +124,17 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes) { std::vector key_columns{"K1", "K2", "K3"}; auto sort_description = getSortDescription(key_columns); - auto pipes = getInputStreamsEqualStride(key_columns, {{1000, 1, 3}, {1500, 1, 3}, {1400, 1, 3}}); + auto pipe = getInputStreamsEqualStride(key_columns, {{1000, 1, 3}, {1500, 1, 3}, {1400, 1, 3}}); - EXPECT_EQ(pipes.size(), 3); + EXPECT_EQ(pipe.numOutputPorts(), 3); - auto transform = std::make_shared(pipes.front().getHeader(), pipes.size(), sort_description, + auto transform = std::make_shared(pipe.getHeader(), pipe.numOutputPorts(), sort_description, DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true); + pipe.addTransform(std::move(transform)); + QueryPipeline pipeline; - pipeline.init(Pipe(std::move(pipes), std::move(transform))); + pipeline.init(std::move(pipe)); pipeline.setMaxThreads(1); auto stream = std::make_shared(std::move(pipeline)); diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 272ad7876d9..986de85d712 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -130,7 +130,7 @@ void SelectStreamFactory::createForShard( auto emplace_local_stream = [&]() { - pipes.emplace_back(createLocalStream(modified_query_ast, header, context, processed_stage).getPipe()); + pipes.emplace_back(QueryPipeline::getPipe(createLocalStream(modified_query_ast, header, context, processed_stage))); }; String modified_query = formattedAST(modified_query_ast); @@ -270,7 +270,7 @@ void SelectStreamFactory::createForShard( } if (try_results.empty() || local_delay < max_remote_delay) - return createLocalStream(modified_query_ast, header, context, stage).getPipe(); + return QueryPipeline::getPipe(createLocalStream(modified_query_ast, header, context, stage)); else { std::vector connections; diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 6f1215d6af0..c0ed5d8a36c 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -89,7 +89,7 @@ BlockIO InterpreterAlterQuery::execute() table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, context.getSettingsRef()); auto partition_commands_pipe = table->alterPartition(query_ptr, metadata_snapshot, partition_commands, context); if (!partition_commands_pipe.empty()) - res.pipeline.init(std::move(partition_commands_pipes)); + res.pipeline.init(std::move(partition_commands_pipe)); } if (!live_view_commands.empty()) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 4fed6f02d5f..9960509a5d7 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -269,7 +269,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() if (settings.graph) { - auto processors = Pipe::detachProcessors(std::move(*pipeline).getPipe()); + auto processors = Pipe::detachProcessors(QueryPipeline::getPipe(std::move(*pipeline))); if (settings.compact) printPipelineCompact(processors, buffer, settings.query_pipeline_options.header); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 36b106d5d68..31a623e82fd 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -190,7 +190,7 @@ BlockIO InterpreterInsertQuery::execute() } } - res.pipeline.unitePipelines(std::move(pipelines), {}); + res.pipeline = QueryPipeline::unitePipelines(std::move(pipelines), {}); } } diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index 7a0c7184882..e6749827efe 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -11,41 +11,12 @@ namespace DB { + namespace ErrorCodes { extern const int LOGICAL_ERROR; } -static void checkSingleInput(const IProcessor & transform) -{ - if (transform.getInputs().size() != 1) - throw Exception("Processor for pipe should have single input, " - "but " + transform.getName() + " has " + - toString(transform.getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR); -} - -static void checkMultipleInputs(const IProcessor & transform, size_t num_inputs) -{ - if (transform.getInputs().size() != num_inputs) - throw Exception("Processor for pipe should have " + toString(num_inputs) + " inputs, " - "but " + transform.getName() + " has " + - toString(transform.getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR); -} - -static void checkSingleOutput(const IProcessor & transform) -{ - if (transform.getOutputs().size() != 1) - throw Exception("Processor for pipe should have single output, " - "but " + transform.getName() + " has " + - toString(transform.getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); -} - -static void checkSimpleTransform(const IProcessor & transform) -{ - checkSingleInput(transform); - checkSingleOutput(transform); -} - static void checkSource(const IProcessor & source) { if (!source.getInputs().empty()) @@ -125,6 +96,16 @@ static OutputPort * uniteTotals(const OutputPortRawPtrs & ports, const Block & h return totals_port; } +Pipe::Holder & Pipe::Holder::operator=(Holder && rhs) +{ + table_locks.insert(table_locks.end(), rhs.table_locks.begin(), rhs.table_locks.end()); + storage_holders.insert(storage_holders.end(), rhs.storage_holders.begin(), rhs.storage_holders.end()); + interpreter_context.insert(interpreter_context.end(), + rhs.interpreter_context.begin(), rhs.interpreter_context.end()); + + return *this; +} + Pipe::Pipe(ProcessorPtr source, OutputPort * output, OutputPort * totals, OutputPort * extremes) { if (!source->getInputs().empty()) @@ -190,7 +171,7 @@ Pipe::Pipe(ProcessorPtr source) checkSource(*source); if (collected_processors) - collected_processors->emplace_back(source.get()); + collected_processors->emplace_back(source); output_ports.push_back(&source->getOutputs().front()); header = output_ports.front()->getHeader(); @@ -248,7 +229,7 @@ Pipe::Pipe(Processors processors_) : processors(std::move(processors_)) if (collected_processors) for (const auto & processor : processors) - collected_processors->emplace_back(processor.get()); + collected_processors->emplace_back(processor); } static Pipes removeEmptyPipes(Pipes pipes) @@ -272,15 +253,22 @@ Pipe Pipe::unitePipes(Pipes pipes) Pipe Pipe::unitePipes(Pipes pipes, Processors * collected_processors) { + Pipe res; + + for (auto & pipe : pipes) + res.holder = std::move(pipe.holder); /// see move assignment for Pipe::Holder. + pipes = removeEmptyPipes(std::move(pipes)); if (pipes.empty()) - return {}; + return res; if (pipes.size() == 1) + { + pipes[0].holder = std::move(res.holder); return std::move(pipes[0]); + } - Pipe res; OutputPortRawPtrs totals; OutputPortRawPtrs extremes; res.header = pipes.front().header; @@ -291,10 +279,6 @@ Pipe Pipe::unitePipes(Pipes pipes, Processors * collected_processors) assertBlocksHaveEqualStructure(res.header, pipe.header, "Pipe::unitePipes"); res.processors.insert(res.processors.end(), pipe.processors.begin(), pipe.processors.end()); res.output_ports.insert(res.output_ports.end(), pipe.output_ports.begin(), pipe.output_ports.end()); - res.table_locks.insert(res.table_locks.end(), pipe.table_locks.begin(), pipe.table_locks.end()); - res.storage_holders.insert(res.storage_holders.end(), pipe.storage_holders.begin(), pipe.storage_holders.end()); - res.interpreter_context.insert(res.interpreter_context.end(), - pipe.interpreter_context.begin(), pipe.interpreter_context.end()); res.max_parallel_streams += pipe.max_parallel_streams; @@ -315,12 +299,14 @@ Pipe Pipe::unitePipes(Pipes pipes, Processors * collected_processors) for (; num_processors < res.processors.size(); ++num_processors) res.collected_processors->emplace_back(res.processors[num_processors]); } + + return res; } void Pipe::addSource(ProcessorPtr source) { checkSource(*source); - const auto & source_header = output_ports.front()->getHeader(); + const auto & source_header = source->getOutputs().front().getHeader(); if (output_ports.empty()) header = source_header; @@ -328,7 +314,7 @@ void Pipe::addSource(ProcessorPtr source) assertBlocksHaveEqualStructure(header, source_header, "Pipes"); if (collected_processors) - collected_processors->emplace_back(source.get()); + collected_processors->emplace_back(source); output_ports.push_back(&source->getOutputs().front()); processors.emplace_back(std::move(source)); @@ -350,7 +336,7 @@ void Pipe::addTotalsSource(ProcessorPtr source) assertBlocksHaveEqualStructure(header, source_header, "Pipes"); if (collected_processors) - collected_processors->emplace_back(source.get()); + collected_processors->emplace_back(source); totals_port = &source->getOutputs().front(); processors.emplace_back(std::move(source)); @@ -370,7 +356,7 @@ void Pipe::addExtremesSource(ProcessorPtr source) assertBlocksHaveEqualStructure(header, source_header, "Pipes"); if (collected_processors) - collected_processors->emplace_back(source.get()); + collected_processors->emplace_back(source); extremes_port = &source->getOutputs().front(); processors.emplace_back(std::move(source)); @@ -385,7 +371,7 @@ static void dropPort(OutputPort *& port, Processors & processors, Processors * c connect(*port, null_sink->getPort()); if (collected_processors) - collected_processors->emplace_back(null_sink.get()); + collected_processors->emplace_back(null_sink); processors.emplace_back(std::move(null_sink)); port = nullptr; @@ -471,14 +457,15 @@ void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort for (size_t i = 1; i < output_ports.size(); ++i) assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipes"); - if (totals_port) - assertBlocksHaveEqualStructure(header, totals_port->getHeader(), "Pipes"); + // Temporarily skip this check. TotaslHavingTransform may return finalized totals but not finalized data. + // if (totals_port) + // assertBlocksHaveEqualStructure(header, totals_port->getHeader(), "Pipes"); if (extremes_port) assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); if (collected_processors) - collected_processors->emplace_back(transform.get()); + collected_processors->emplace_back(transform); processors.emplace_back(std::move(transform)); @@ -526,7 +513,7 @@ void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter) port = &transform->getOutputs().front(); if (collected_processors) - collected_processors->emplace_back(transform.get()); + collected_processors->emplace_back(transform); processors.emplace_back(std::move(transform)); } @@ -614,7 +601,7 @@ void Pipe::setOutputFormat(ProcessorPtr output) addExtremesSource(std::make_shared(extremes.getHeader())); if (collected_processors) - collected_processors->emplace_back(output.get()); + collected_processors->emplace_back(output); processors.emplace_back(std::move(output)); @@ -647,7 +634,8 @@ void Pipe::transform(const Transformer & transformer) set.emplace(&port->getProcessor()); } - OutputPortRawPtrs new_output_ports; + output_ports.clear(); + for (const auto & processor : new_processors) { for (const auto & port : processor->getInputs()) @@ -667,7 +655,7 @@ void Pipe::transform(const Transformer & transformer) { if (!port.isConnected()) { - new_output_ports.push_back(&port); + output_ports.push_back(&port); continue; } @@ -696,7 +684,7 @@ void Pipe::transform(const Transformer & transformer) if (collected_processors) { for (const auto & processor : processors) - collected_processors->emplace_back(processor.get()); + collected_processors->emplace_back(processor); } processors.insert(processors.end(), new_processors.begin(), new_processors.end()); diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 787bade065a..6795d3b58b4 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -73,8 +73,8 @@ public: using ProcessorGetterWithStreamKind = std::function; /// Add transform with single input and single output for each port. - void addSimpleTransform(const ProcessorGetter & port); - void addSimpleTransform(const ProcessorGetterWithStreamKind & port); + void addSimpleTransform(const ProcessorGetter & getter); + void addSimpleTransform(const ProcessorGetterWithStreamKind & getter); using Transformer = std::function; @@ -92,20 +92,30 @@ public: void setQuota(const std::shared_ptr & quota); /// Do not allow to change the table while the processors of pipe are alive. - void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } + void addTableLock(const TableLockHolder & lock) { holder.table_locks.push_back(lock); } /// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible. - void addInterpreterContext(std::shared_ptr context) { interpreter_context.emplace_back(std::move(context)); } - void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); } + void addInterpreterContext(std::shared_ptr context) { holder.interpreter_context.emplace_back(std::move(context)); } + void addStorageHolder(StoragePtr storage) { holder.storage_holders.emplace_back(std::move(storage)); } private: /// Destruction order: processors, header, locks, temporary storages, local contexts - /// Some processors may implicitly use Context or temporary Storage created by Interpreter. - /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, - /// because QueryPipeline is alive until query is finished. - std::vector> interpreter_context; - std::vector storage_holders; - std::vector table_locks; + struct Holder + { + Holder() = default; + Holder(Holder &&) = default; + /// Custom mode assignment does not destroy data from lhs. It appends data from rhs to lhs. + Holder& operator=(Holder &&); + + /// Some processors may implicitly use Context or temporary Storage created by Interpreter. + /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, + /// because QueryPipeline is alive until query is finished. + std::vector> interpreter_context; + std::vector storage_holders; + std::vector table_locks; + }; + + Holder holder; /// Header is common for all output below. Block header; diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 97e96387f7f..751f93ad694 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -66,12 +66,18 @@ void QueryPipeline::init(Pipe pipe_) if (initialized()) throw Exception("Pipeline has already been initialized.", ErrorCodes::LOGICAL_ERROR); - if (pipe.empty()) + if (pipe_.empty()) throw Exception("Can't initialize pipeline with empty pipe.", ErrorCodes::LOGICAL_ERROR); pipe = std::move(pipe_); } +void QueryPipeline::reset() +{ + Pipe pipe_to_destroy(std::move(pipe)); + *this = QueryPipeline(); +} + void QueryPipeline::addSimpleTransform(const Pipe::ProcessorGetter & getter) { checkInitializedAndNotCompleted(); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 74cc692ac42..385cf77198e 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -33,6 +33,9 @@ public: /// All pipes must have same header. void init(Pipe pipe); + /// Clear and release all resources. + void reset(); + bool initialized() { return !pipe.empty(); } bool isCompleted() { return pipe.isCompleted(); } diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.cpp b/src/Processors/QueryPlan/ReadFromStorageStep.cpp index 39b0743fe38..3af011d1333 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromStorageStep.cpp @@ -120,8 +120,6 @@ ReadFromStorageStep::ReadFromStorageStep( pipe.setQuota(quota); } - pipe.enableQuota(); - pipeline->init(std::move(pipe)); pipeline->addInterpreterContext(std::move(context)); diff --git a/src/Processors/QueryPlan/UnionStep.cpp b/src/Processors/QueryPlan/UnionStep.cpp index b645eb3f3d7..1e74046b071 100644 --- a/src/Processors/QueryPlan/UnionStep.cpp +++ b/src/Processors/QueryPlan/UnionStep.cpp @@ -30,7 +30,7 @@ QueryPipelinePtr UnionStep::updatePipeline(QueryPipelines pipelines) return pipeline; } - pipeline->unitePipelines(std::move(pipelines), output_stream->header ,max_threads); + *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), output_stream->header ,max_threads); processors = collector.detachProcessors(); return pipeline; diff --git a/src/Storages/LiveView/StorageBlocks.h b/src/Storages/LiveView/StorageBlocks.h index 56fd0c620c2..37861b55568 100644 --- a/src/Storages/LiveView/StorageBlocks.h +++ b/src/Storages/LiveView/StorageBlocks.h @@ -35,7 +35,7 @@ public: QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const ASTPtr &) const override { return to_stage; } - Pipes read( + Pipe read( const Names & /*column_names*/, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & /*query_info*/, @@ -44,7 +44,7 @@ public: size_t /*max_block_size*/, unsigned /*num_streams*/) override { - return std::move(pipes); + return Pipe::unitePipes(std::move(pipes)); } private: diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index f2049a9c248..202603b7481 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -26,14 +26,12 @@ #include #include #include -#include #include #include #include #include #include -#include #include #include @@ -730,8 +728,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (metadata_snapshot->hasSortingKey()) { - auto expr = std::make_shared(pipe.getHeader(), metadata_snapshot->getSortingKey().expression); - pipe.addSimpleTransform(std::move(expr)); + pipe.addSimpleTransform([&metadata_snapshot](const Block & header) + { + return std::make_shared(header, metadata_snapshot->getSortingKey().expression); + }); } pipes.emplace_back(std::move(pipe)); @@ -800,7 +800,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor } QueryPipeline pipeline; - pipeline.init(Pipe(std::move(pipes), std::move(merged_transform))); + pipeline.init(Pipe::unitePipes(std::move(pipes))); + pipeline.addTransform(std::move(merged_transform)); pipeline.setMaxThreads(1); BlockInputStreamPtr merged_stream = std::make_shared(std::move(pipeline)); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index b2d00be8620..6109c7ace6d 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1122,7 +1122,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( auto pipe = Pipe::unitePipes(std::move(pipes)); - if (input_order_info->direction == 1) + if (input_order_info->direction != 1) { pipe.addSimpleTransform([](const Block & header) { @@ -1290,9 +1290,9 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( key_columns.emplace_back(desc.column_number); } - pipe.addSimpleTransform([&](const Block & header) + pipe.addSimpleTransform([&](const Block & stream_header) { - return std::make_shared(header, num_streams, key_columns); + return std::make_shared(stream_header, num_streams, key_columns); }); pipe.transform([&](OutputPortRawPtrs ports) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index a3d16087e34..6052837b2cc 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -139,7 +139,7 @@ void StorageRabbitMQ::loopingFunc() } -Pipes StorageRabbitMQ::read( +Pipe StorageRabbitMQ::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /* query_info */, @@ -171,7 +171,7 @@ Pipes StorageRabbitMQ::read( } LOG_DEBUG(log, "Starting reading {} streams", pipes.size()); - return pipes; + return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index e1c8b33c91e..064af15c16c 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -31,7 +31,7 @@ public: void startup() override; void shutdown() override; - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 4a728841c50..5b9957f4ed4 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -221,10 +221,10 @@ Pipe StorageBuffer::read( columns_intersection, destination_metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - pipe_from_dst.addSimpleTransform([&](const Block & header) + pipe_from_dst.addSimpleTransform([&](const Block & stream_header) { - return std::make_shared( - header, header_after_adding_defaults, metadata_snapshot->getColumns().getDefaults(), context); + return std::make_shared(stream_header, header_after_adding_defaults, + metadata_snapshot->getColumns().getDefaults(), context); }); pipe_from_dst.addSimpleTransform([&](const Block & stream_header) @@ -252,7 +252,7 @@ Pipe StorageBuffer::read( if (!pipe_from_buffers.empty() && !pipe_from_dst.empty() && !blocksHaveEqualStructure(pipe_from_buffers.getHeader(), pipe_from_dst.getHeader())) { - pipe_from_buffers.addSimpleTransform([&](const Block & header) + pipe_from_dst.addSimpleTransform([&](const Block & header) { return std::make_shared( header, @@ -265,7 +265,9 @@ Pipe StorageBuffer::read( * then sources from the buffers must also be wrapped in the processing pipeline before the same stage. */ if (processed_stage > QueryProcessingStage::FetchColumns) - pipe_from_buffers = InterpreterSelectQuery(query_info.query, context, std::move(pipe_from_buffers), SelectQueryOptions(processed_stage)).execute().pipeline.getPipe(); + pipe_from_buffers = QueryPipeline::getPipe( + InterpreterSelectQuery(query_info.query, context, std::move(pipe_from_buffers), + SelectQueryOptions(processed_stage)).execute().pipeline); if (query_info.prewhere_info) { @@ -285,7 +287,10 @@ Pipe StorageBuffer::read( } } - return Pipe::unitePipes({std::move(pipe_from_dst), std::move(pipe_from_buffers)}); + Pipes pipes; + pipes.emplace_back(std::move(pipe_from_dst)); + pipes.emplace_back(std::move(pipe_from_buffers)); + return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 471a9f522ab..558216a6216 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -419,7 +419,7 @@ Pipe StorageFile::read( pipes.emplace_back(std::make_shared( this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults())); - return Pipe::unitePipes(pipes); + return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 8020c54d8cd..62d8259f705 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -467,7 +467,7 @@ Pipe StorageGenerateRandom::read( for (UInt64 i = 0; i < num_streams; ++i) pipes.emplace_back(std::make_shared(max_block_size, max_array_length, max_string_length, generate(), block_header, context)); - return Pipe::unitePipes(pipes); + return Pipe::unitePipes(std::move(pipes)); } } diff --git a/src/Storages/StorageHDFS.cpp b/src/Storages/StorageHDFS.cpp index 98106850492..65b1eed7793 100644 --- a/src/Storages/StorageHDFS.cpp +++ b/src/Storages/StorageHDFS.cpp @@ -298,7 +298,7 @@ Pipe StorageHDFS::read( pipes.emplace_back(std::make_shared( sources_info, uri_without_path, format_name, compression_method, metadata_snapshot->getSampleBlock(), context_, max_block_size)); - return Pipe::unitePipes(pipes); + return Pipe::unitePipes(std::move(pipes)); } BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 50b31a1f128..042f3abff4e 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -647,7 +647,7 @@ Pipe StorageLog::read( max_read_buffer_size)); } - return Pipe::unitePipes(pipes); + return Pipe::unitePipes(std::move(pipes)); } BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 1c690221967..e9881fe4804 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -132,7 +132,7 @@ Pipe StorageMemory::read( pipes.emplace_back(std::make_shared(column_names, begin, end, *this, metadata_snapshot)); } - return Pipe::unitePipes(pipes); + return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 15e9fd65448..a98d789a048 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -136,7 +136,7 @@ Pipe StorageMerge::read( const size_t max_block_size, unsigned num_streams) { - Pipe pipe; + Pipes pipes; bool has_table_virtual_column = false; Names real_column_names; @@ -210,12 +210,16 @@ Pipe StorageMerge::read( auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); - pipe = createSources( + auto source_pipe = createSources( storage_metadata_snapshot, query_info, processed_stage, max_block_size, header, table, real_column_names, modified_context, current_streams, has_table_virtual_column); + + pipes.emplace_back(std::move(source_pipe)); } + auto pipe = Pipe::unitePipes(std::move(pipes)); + if (!pipe.empty()) narrowPipe(pipe, num_streams); @@ -245,10 +249,10 @@ Pipe StorageMerge::createSources( if (!storage) { - pipe = InterpreterSelectQuery( + pipe = QueryPipeline::getPipe(InterpreterSelectQuery( modified_query_info.query, *modified_context, std::make_shared(header), - SelectQueryOptions(processed_stage).analyze()).execute().pipeline.getPipe(); + SelectQueryOptions(processed_stage).analyze()).execute().pipeline); pipe.addInterpreterContext(modified_context); return pipe; @@ -275,13 +279,13 @@ Pipe StorageMerge::createSources( InterpreterSelectQuery interpreter{modified_query_info.query, *modified_context, SelectQueryOptions(processed_stage)}; - pipe = interpreter.execute().pipeline.getPipe(); + pipe = QueryPipeline::getPipe(interpreter.execute().pipeline); /** Materialization is needed, since from distributed storage the constants come materialized. * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, * And this is not allowed, since all code is based on the assumption that in the block stream all types are the same. */ - pipe.addSimpleTransform([](const Block & header) { return std::make_shared(header); }); + pipe.addSimpleTransform([](const Block & stream_header) { return std::make_shared(stream_header); }); } if (!pipe.empty()) @@ -291,10 +295,10 @@ Pipe StorageMerge::createSources( if (has_table_virtual_column) { - pipe.addSimpleTransform([name = table_name](const Block & header) + pipe.addSimpleTransform([name = table_name](const Block & stream_header) { return std::make_shared>( - header, std::make_shared(), name, "_table"); + stream_header, std::make_shared(), name, "_table"); }); } @@ -440,9 +444,9 @@ void StorageMerge::convertingSourceStream( QueryProcessingStage::Enum processed_stage) { Block before_block_header = pipe.getHeader(); - pipe.addSimpleTransform([&before_block_header](const Block & header) + pipe.addSimpleTransform([&](const Block & stream_header) { - return std::make_shared(before_block_header, header, ConvertingTransform::MatchColumnsMode::Name); + return std::make_shared(stream_header, header, ConvertingTransform::MatchColumnsMode::Name); }); auto where_expression = query->as()->where(); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 4fa2eaed27d..5e2d56d18c0 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -82,7 +82,7 @@ protected: const Context & context, QueryProcessingStage::Enum processed_stage); - Pipes createSources( + Pipe createSources( const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage, diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 66d7c79e434..e4ba104cffc 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -310,7 +310,7 @@ Pipe StorageStripeLog::read( String index_file = table_path + "index.mrk"; if (!disk->exists(index_file)) { - Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); + return Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); } CompressedReadBufferFromFile index_in(disk->readFile(index_file, INDEX_BUFFER_SIZE)); @@ -334,7 +334,7 @@ Pipe StorageStripeLog::read( /// We do not keep read lock directly at the time of reading, because we read ranges of data that do not change. - return Pipe::unitePipes(pipes); + return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 2c68be4fdb1..1a95b7ea21f 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -86,7 +86,7 @@ Pipe StorageView::read( column_names, getVirtuals(), getStorageID()), ConvertingTransform::MatchColumnsMode::Name); }); - return std::move(pipeline).getPipe(); + return QueryPipeline::getPipe(std::move(pipeline)); } static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query) diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 7c2ef85f158..b30d6476b22 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -28,7 +28,7 @@ public: setInMemoryMetadata(metadata_); } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -46,10 +46,7 @@ public: UInt64 num_rows = res_columns.at(0)->size(); Chunk chunk(std::move(res_columns), num_rows); - Pipes pipes; - pipes.emplace_back(std::make_shared(sample_block, std::move(chunk))); - - return pipes; + return Pipe(std::make_shared(sample_block, std::move(chunk))); } }; diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 57d8d13e79b..18cc46c0c1b 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -240,7 +240,7 @@ private: }; -Pipes StorageSystemColumns::read( +Pipe StorageSystemColumns::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -294,7 +294,7 @@ Pipes StorageSystemColumns::read( if (!block_to_filter.rows()) { pipes.emplace_back(std::make_shared(header)); - return pipes; + return Pipe::unitePipes(std::move(pipes)); } ColumnPtr & database_column = block_to_filter.getByName("database").column; @@ -333,7 +333,7 @@ Pipes StorageSystemColumns::read( if (!block_to_filter.rows()) { pipes.emplace_back(std::make_shared(header)); - return pipes; + return Pipe::unitePipes(std::move(pipes)); } ColumnPtr filtered_database_column = block_to_filter.getByName("database").column; @@ -344,7 +344,7 @@ Pipes StorageSystemColumns::read( std::move(filtered_database_column), std::move(filtered_table_column), std::move(storages), context)); - return pipes; + return Pipe::unitePipes(std::move(pipes)); } } diff --git a/src/Storages/System/StorageSystemColumns.h b/src/Storages/System/StorageSystemColumns.h index 7336b406183..480afe5c36c 100644 --- a/src/Storages/System/StorageSystemColumns.h +++ b/src/Storages/System/StorageSystemColumns.h @@ -17,7 +17,7 @@ class StorageSystemColumns final : public ext::shared_ptr_helpersize(); Chunk chunk(std::move(new_columns), num_rows); - Pipes pipes; - pipes.emplace_back(std::make_shared(std::move(block), std::move(chunk))); - return pipes; + return Pipe(std::make_shared(std::move(block), std::move(chunk))); } }; diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index 554b8cfd1eb..b54f3510eb6 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -27,7 +27,7 @@ StorageSystemDisks::StorageSystemDisks(const std::string & name_) setInMemoryMetadata(storage_metadata); } -Pipes StorageSystemDisks::read( +Pipe StorageSystemDisks::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & /*query_info*/, @@ -66,10 +66,7 @@ Pipes StorageSystemDisks::read( UInt64 num_rows = res_columns.at(0)->size(); Chunk chunk(std::move(res_columns), num_rows); - Pipes pipes; - pipes.emplace_back(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); - - return pipes; + return Pipe(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); } } diff --git a/src/Storages/System/StorageSystemDisks.h b/src/Storages/System/StorageSystemDisks.h index 714a0a5428c..e2fb0e03705 100644 --- a/src/Storages/System/StorageSystemDisks.h +++ b/src/Storages/System/StorageSystemDisks.h @@ -20,7 +20,7 @@ class StorageSystemDisks final : public ext::shared_ptr_helper #include #include -#include #include #include @@ -123,7 +122,7 @@ StorageSystemNumbers::StorageSystemNumbers(const StorageID & table_id, bool mult setInMemoryMetadata(storage_metadata); } -Pipes StorageSystemNumbers::read( +Pipe StorageSystemNumbers::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo &, @@ -143,8 +142,7 @@ Pipes StorageSystemNumbers::read( if (!multithreaded) num_streams = 1; - Pipes res; - res.reserve(num_streams); + Pipe pipe; if (num_streams > 1 && !even_distribution && *limit) { @@ -152,9 +150,9 @@ Pipes StorageSystemNumbers::read( UInt64 max_counter = offset + *limit; for (size_t i = 0; i < num_streams; ++i) - res.emplace_back(std::make_shared(state, max_block_size, max_counter)); + pipe.addSource(std::make_shared(state, max_block_size, max_counter)); - return res; + return pipe; } for (size_t i = 0; i < num_streams; ++i) @@ -164,17 +162,22 @@ Pipes StorageSystemNumbers::read( if (limit && i == 0) source->addTotalRowsApprox(*limit); - res.emplace_back(std::move(source)); - - if (limit) - { - /// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly. - res.back().addSimpleTransform(std::make_shared( - res.back().getHeader(), *limit * (i + 1) / num_streams - *limit * i / num_streams, 0)); - } + pipe.addSource(std::move(source)); } - return res; + if (limit) + { + size_t i = 0; + /// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly. + pipe.addSimpleTransform([&](const Block & header) + { + ++i; + return std::make_shared( + header, *limit * i / num_streams - *limit * (i - 1) / num_streams, 0); + }); + } + + return pipe; } } diff --git a/src/Storages/System/StorageSystemNumbers.h b/src/Storages/System/StorageSystemNumbers.h index 88d3651c7f9..f907f3d5f93 100644 --- a/src/Storages/System/StorageSystemNumbers.h +++ b/src/Storages/System/StorageSystemNumbers.h @@ -29,7 +29,7 @@ class StorageSystemNumbers final : public ext::shared_ptr_helperconvertToFullColumnIfConst(); Chunk chunk({ std::move(column) }, 1); - Pipes pipes; - pipes.emplace_back(std::make_shared(std::move(header), std::move(chunk))); - - return pipes; + return Pipe(std::make_shared(std::move(header), std::move(chunk))); } diff --git a/src/Storages/System/StorageSystemOne.h b/src/Storages/System/StorageSystemOne.h index 3b3e531dc86..09db9822914 100644 --- a/src/Storages/System/StorageSystemOne.h +++ b/src/Storages/System/StorageSystemOne.h @@ -21,7 +21,7 @@ class StorageSystemOne final : public ext::shared_ptr_helper, public: std::string getName() const override { return "SystemOne"; } - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 168b305605d..b6e94993468 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -223,7 +223,7 @@ StoragesInfo StoragesInfoStream::next() return {}; } -Pipes StorageSystemPartsBase::read( +Pipe StorageSystemPartsBase::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -254,10 +254,7 @@ Pipes StorageSystemPartsBase::read( UInt64 num_rows = res_columns.at(0)->size(); Chunk chunk(std::move(res_columns), num_rows); - Pipes pipes; - pipes.emplace_back(std::make_shared(std::move(header), std::move(chunk))); - - return pipes; + return Pipe(std::make_shared(std::move(header), std::move(chunk))); } diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 56c9a8fb0d0..040a03770ac 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -55,7 +55,7 @@ private: class StorageSystemPartsBase : public IStorage { public: - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 27a9cd0c4bb..f049b69a7ff 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -57,7 +57,7 @@ StorageSystemReplicas::StorageSystemReplicas(const std::string & name_) } -Pipes StorageSystemReplicas::read( +Pipe StorageSystemReplicas::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -139,7 +139,7 @@ Pipes StorageSystemReplicas::read( VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); if (!filtered_block.rows()) - return Pipes(); + return {}; col_database = filtered_block.getByName("database").column; col_table = filtered_block.getByName("table").column; @@ -202,9 +202,7 @@ Pipes StorageSystemReplicas::read( UInt64 num_rows = fin_columns.at(0)->size(); Chunk chunk(std::move(fin_columns), num_rows); - Pipes pipes; - pipes.emplace_back(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); - return pipes; + return Pipe(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); } diff --git a/src/Storages/System/StorageSystemReplicas.h b/src/Storages/System/StorageSystemReplicas.h index b068ebc8b0a..bc830721a23 100644 --- a/src/Storages/System/StorageSystemReplicas.h +++ b/src/Storages/System/StorageSystemReplicas.h @@ -18,7 +18,7 @@ class StorageSystemReplicas final : public ext::shared_ptr_helpersize(); Chunk chunk(std::move(res_columns), num_rows); - Pipes pipes; - pipes.emplace_back(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); - - return pipes; + return Pipe(std::make_shared(metadata_snapshot->getSampleBlock(), std::move(chunk))); } } diff --git a/src/Storages/System/StorageSystemStoragePolicies.h b/src/Storages/System/StorageSystemStoragePolicies.h index a1427da8559..73fda1e8c46 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.h +++ b/src/Storages/System/StorageSystemStoragePolicies.h @@ -20,7 +20,7 @@ class StorageSystemStoragePolicies final : public ext::shared_ptr_helper( + return Pipe(std::make_shared( std::move(columns_mask), std::move(res_block), max_block_size, std::move(filtered_databases_column), context)); - - return pipes; } } diff --git a/src/Storages/System/StorageSystemTables.h b/src/Storages/System/StorageSystemTables.h index 54551205684..766c3046dbc 100644 --- a/src/Storages/System/StorageSystemTables.h +++ b/src/Storages/System/StorageSystemTables.h @@ -18,7 +18,7 @@ class StorageSystemTables final : public ext::shared_ptr_helperaddTotalRowsApprox(*limit); - res.emplace_back(std::move(source)); + res.addSource(std::move(source)); } return res; diff --git a/src/Storages/System/StorageSystemZeros.h b/src/Storages/System/StorageSystemZeros.h index f169861122a..41de3ce6246 100644 --- a/src/Storages/System/StorageSystemZeros.h +++ b/src/Storages/System/StorageSystemZeros.h @@ -20,7 +20,7 @@ class StorageSystemZeros final : public ext::shared_ptr_helper Date: Thu, 6 Aug 2020 15:30:51 +0300 Subject: [PATCH 127/374] try fix synchronization check --- src/Core/ya.make | 2 ++ src/DataStreams/ya.make | 1 + src/Databases/MySQL/MaterializeMySQLSyncThread.h | 4 +++- src/Databases/ya.make | 6 +++++- .../MySQL/tests/gtest_create_rewritten.cpp | 1 - src/Interpreters/ya.make | 2 ++ src/Parsers/ya.make | 14 ++++++++++++++ src/Storages/ya.make | 1 + 8 files changed, 28 insertions(+), 3 deletions(-) diff --git a/src/Core/ya.make b/src/Core/ya.make index 197908300fe..3b45cf22f57 100644 --- a/src/Core/ya.make +++ b/src/Core/ya.make @@ -17,7 +17,9 @@ SRCS( ExternalTable.cpp Field.cpp iostream_debug_helpers.cpp + MySQLClient.cpp MySQLProtocol.cpp + MySQLReplication.cpp NamesAndTypes.cpp PostgreSQLProtocol.cpp Settings.cpp diff --git a/src/DataStreams/ya.make b/src/DataStreams/ya.make index a9bbd987ff0..dcada250d38 100644 --- a/src/DataStreams/ya.make +++ b/src/DataStreams/ya.make @@ -11,6 +11,7 @@ NO_COMPILER_WARNINGS() SRCS( AddingDefaultBlockOutputStream.cpp AddingDefaultsBlockInputStream.cpp + AddingVersionsBlockOutputStream.cpp AsynchronousBlockInputStream.cpp BlockIO.cpp BlockStreamProfileInfo.cpp diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index f244c24ab9a..9c454de2a3f 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -1,6 +1,8 @@ #pragma once -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif #if USE_MYSQL diff --git a/src/Databases/ya.make b/src/Databases/ya.make index 1fc21caf340..6af786578f8 100644 --- a/src/Databases/ya.make +++ b/src/Databases/ya.make @@ -11,11 +11,15 @@ SRCS( DatabaseFactory.cpp DatabaseLazy.cpp DatabaseMemory.cpp - DatabaseMySQL.cpp DatabaseOnDisk.cpp DatabaseOrdinary.cpp DatabasesCommon.cpp DatabaseWithDictionaries.cpp + MySQL/DatabaseConnectionMySQL.cpp + MySQL/DatabaseMaterializeMySQL.cpp + MySQL/MaterializeMetadata.cpp + MySQL/MaterializeMySQLSettings.cpp + MySQL/MaterializeMySQLSyncThread.cpp ) diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index 62683dc546d..9fb71cf3a14 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -22,7 +22,6 @@ static inline ASTPtr tryRewrittenCreateQuery(const String & query, const Context ASTPtr ast = parseQuery(external_ddl_parser, query, 0, 0); Context context = context_; - context.unsafeSetCurrentDatabase("default"); return MySQLInterpreter::InterpreterCreateImpl::getRewrittenQuery( *ast->as()->external_ddl->as(), context, "test_database", "test_database"); diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index b543319dfa3..23cde61a744 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -78,6 +78,7 @@ SRCS( InterpreterDropQuery.cpp InterpreterExistsQuery.cpp InterpreterExplainQuery.cpp + InterpreterExternalDDLQuery.cpp InterpreterFactory.cpp InterpreterGrantQuery.cpp InterpreterInsertQuery.cpp @@ -110,6 +111,7 @@ SRCS( MergeJoin.cpp MetricLog.cpp MutationsInterpreter.cpp + MySQL/InterpretersMySQLDDLQuery.cpp NullableUtils.cpp OptimizeIfChains.cpp OptimizeIfWithConstantConditionVisitor.cpp diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make index ada5975bcf0..a2cc84223a7 100644 --- a/src/Parsers/ya.make +++ b/src/Parsers/ya.make @@ -66,6 +66,19 @@ SRCS( IParserBase.cpp Lexer.cpp makeASTForLogicalFunction.cpp + MySQL/ASTAlterCommand.cpp + MySQL/ASTAlterQuery.cpp + MySQL/ASTCreateDefines.cpp + MySQL/ASTCreateQuery.cpp + MySQL/ASTDeclareColumn.cpp + MySQL/ASTDeclareConstraint.cpp + MySQL/ASTDeclareIndex.cpp + MySQL/ASTDeclareOption.cpp + MySQL/ASTDeclarePartition.cpp + MySQL/ASTDeclarePartitionOptions.cpp + MySQL/ASTDeclareReference.cpp + MySQL/ASTDeclareSubPartition.cpp + MySQL/ASTDeclareTableOptions.cpp parseDatabaseAndTableName.cpp parseIdentifierOrStringLiteral.cpp parseIntervalKind.cpp @@ -86,6 +99,7 @@ SRCS( ParserDropAccessEntityQuery.cpp ParserDropQuery.cpp ParserExplainQuery.cpp + ParserExternalDDLQuery.cpp ParserGrantQuery.cpp ParserInsertQuery.cpp ParserKillQueryQuery.cpp diff --git a/src/Storages/ya.make b/src/Storages/ya.make index a6c7fc5e4fe..8b60bd34a36 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -115,6 +115,7 @@ SRCS( StorageLog.cpp StorageLogSettings.cpp StorageMaterializedView.cpp + StorageMaterializeMySQL.cpp StorageMemory.cpp StorageMerge.cpp StorageMergeTree.cpp From caef062135c24174159223eda0d73bb7d3a5a3e8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 6 Aug 2020 15:34:16 +0300 Subject: [PATCH 128/374] Possibly fix race in StorageMemory. --- src/Storages/StorageMemory.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 3a10b2c7e7d..c48029313f4 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -31,7 +31,7 @@ public: : SourceWithProgress(metadata_snapshot->getSampleBlockForColumns(column_names_, storage.getVirtuals(), storage.getStorageID())) , column_names(std::move(column_names_)) , begin(begin_) - , end(end_) + , end(end_) /// [begin, end] , it(begin) { } @@ -41,7 +41,7 @@ public: protected: Chunk generate() override { - if (it == end) + if (finished) { return {}; } @@ -55,7 +55,10 @@ protected: for (const auto & name : column_names) columns.emplace_back(src.getByName(name).column); - ++it; + if (it == end) + finished = true; + else + ++it; return Chunk(std::move(columns), src.rows()); } } @@ -64,6 +67,7 @@ private: BlocksList::iterator begin; BlocksList::iterator end; BlocksList::iterator it; + bool finished = false; }; @@ -129,6 +133,11 @@ Pipes StorageMemory::read( std::advance(begin, stream * size / num_streams); std::advance(end, (stream + 1) * size / num_streams); + if (begin == end) + continue; + else + --end; + pipes.emplace_back(std::make_shared(column_names, begin, end, *this, metadata_snapshot)); } From 285f3236535d2a3b2ce88a28708346c5fb0a1f29 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 6 Aug 2020 16:22:17 +0300 Subject: [PATCH 129/374] Fix build. --- src/Storages/StorageMemory.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index c48029313f4..b26b41c608e 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -41,7 +41,7 @@ public: protected: Chunk generate() override { - if (finished) + if (is_finished) { return {}; } @@ -56,7 +56,7 @@ protected: columns.emplace_back(src.getByName(name).column); if (it == end) - finished = true; + is_finished = true; else ++it; return Chunk(std::move(columns), src.rows()); @@ -67,7 +67,7 @@ private: BlocksList::iterator begin; BlocksList::iterator end; BlocksList::iterator it; - bool finished = false; + bool is_finished = false; }; From ac52726f7dfa19e3cae8df74175beac00ce2caf7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 6 Aug 2020 17:16:52 +0300 Subject: [PATCH 130/374] Renamed iterators. Added comment. --- src/Storages/StorageMemory.cpp | 37 +++++++++++++++++----------------- 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index b26b41c608e..e098a5ace4e 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -22,17 +22,19 @@ namespace ErrorCodes class MemorySource : public SourceWithProgress { public: + /// We use range [first, last] which includes right border. + /// It is needed because new elements may be added to list in other thread. + /// Appending of new element changes end() of std::list, which may cause data race otherwise. MemorySource( Names column_names_, - BlocksList::iterator begin_, - BlocksList::iterator end_, + BlocksList::iterator first_, + BlocksList::iterator last_, const StorageMemory & storage, const StorageMetadataPtr & metadata_snapshot) : SourceWithProgress(metadata_snapshot->getSampleBlockForColumns(column_names_, storage.getVirtuals(), storage.getStorageID())) , column_names(std::move(column_names_)) - , begin(begin_) - , end(end_) /// [begin, end] - , it(begin) + , first(first_) + , last(last_) /// [first, last] { } @@ -47,7 +49,7 @@ protected: } else { - Block src = *it; + Block src = *first; Columns columns; columns.reserve(column_names.size()); @@ -55,18 +57,17 @@ protected: for (const auto & name : column_names) columns.emplace_back(src.getByName(name).column); - if (it == end) + if (first == last) is_finished = true; else - ++it; + ++first; return Chunk(std::move(columns), src.rows()); } } private: Names column_names; - BlocksList::iterator begin; - BlocksList::iterator end; - BlocksList::iterator it; + BlocksList::iterator first; + BlocksList::iterator last; bool is_finished = false; }; @@ -127,18 +128,18 @@ Pipes StorageMemory::read( for (size_t stream = 0; stream < num_streams; ++stream) { - BlocksList::iterator begin = data.begin(); - BlocksList::iterator end = data.begin(); + BlocksList::iterator first = data.begin(); + BlocksList::iterator last = data.begin(); - std::advance(begin, stream * size / num_streams); - std::advance(end, (stream + 1) * size / num_streams); + std::advance(first, stream * size / num_streams); + std::advance(last, (stream + 1) * size / num_streams); - if (begin == end) + if (first == last) continue; else - --end; + --last; - pipes.emplace_back(std::make_shared(column_names, begin, end, *this, metadata_snapshot)); + pipes.emplace_back(std::make_shared(column_names, first, last, *this, metadata_snapshot)); } return pipes; From 573cb50d345c0b4540f8a827844d452363eef956 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 6 Aug 2020 18:21:45 +0300 Subject: [PATCH 131/374] first commit --- CHANGELOG.md | 164 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 164 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 54f574cc347..e0ec3c58b5a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,167 @@ + +## ClickHouse release 20.6 + +### ClickHouse release v20.6.2.15-prestable FIXME as compared to v20.5.4.40-stable + +#### New Feature + +* Implementation of PostgreSQL-like ILIKE operator for https://github.com/ClickHouse/ClickHouse/issues/11710. [#12125](https://github.com/ClickHouse/ClickHouse/pull/12125) ([Mike](https://github.com/myrrc)). +* Support RIGHT and FULL JOIN with `set join_algorithm=partial_merge`. Only ALL strictness is supported (ANY, SEMI, ANTI, ASOF are not). [#12118](https://github.com/ClickHouse/ClickHouse/pull/12118) ([Artem Zuikov](https://github.com/4ertus2)). +* Add a function initializedAggregation to initialize an aggregation based on a single value. [#12109](https://github.com/ClickHouse/ClickHouse/pull/12109) ([Guillaume Tassery](https://github.com/YiuRULE)). +* #4006 Support ALTER TABLE ... [ADD|MODIFY] COLUMN ... FIRST. [#12073](https://github.com/ClickHouse/ClickHouse/pull/12073) ([Winter Zhang](https://github.com/zhang2014)). +* add function `parseDateTimeBestEffortUS`. [#12028](https://github.com/ClickHouse/ClickHouse/pull/12028) ([flynn](https://github.com/ucasFL)). +* Switched paths in S3 metadata to relative which allows to handle S3 blobs more easily. [#11892](https://github.com/ClickHouse/ClickHouse/pull/11892) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Initial implementation of `EXPLAIN` query. Syntax: `EXPLAIN SELECT ...`. This fixes [#1118](https://github.com/ClickHouse/ClickHouse/issues/1118). [#11873](https://github.com/ClickHouse/ClickHouse/pull/11873) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Add ORCBlockOutputFormat. [#11662](https://github.com/ClickHouse/ClickHouse/pull/11662) ([Kruglov Pavel](https://github.com/Avogar)). +* Add storage RabbitMQ. [#11069](https://github.com/ClickHouse/ClickHouse/pull/11069) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Added new in-memory format of parts in `MergeTree`-family tables, which stores data in memory. Parts are written on disk at first merge. Part will be created in in-memory format if its size in rows or bytes is below thresholds `min_rows_for_compact_part` and `min_bytes_for_compact_part`. Also optional support of Write-Ahead-Log is available, which is enabled by default and is controlled by setting `in_memory_parts_enable_wal`. [#10697](https://github.com/ClickHouse/ClickHouse/pull/10697) ([Anton Popov](https://github.com/CurtizJ)). + +#### Bug Fix + +* Fix wrong index analysis with functions. It could lead to pruning wrong parts, while reading from `MergeTree` tables. Fixes [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). +* Fix error `Cannot convert column because it is constant but values of constants are different in source and result` for remote queries which use deterministic functions in scope of query, but not deterministic between queries, like `now()`, `now64()`, `randConstant()`. Fixes [#11327](https://github.com/ClickHouse/ClickHouse/issues/11327). [#13075](https://github.com/ClickHouse/ClickHouse/pull/13075) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix rare bug when `ALTER DELETE` and `ALTER MODIFY COLUMN` queries executed simultaneously as a single mutation. Bug leads to an incorrect amount of rows in `count.txt` and as a consequence incorrect data in part. Also, fix a small bug with simultaneous `ALTER RENAME COLUMN` and `ALTER ADD COLUMN`. [#12760](https://github.com/ClickHouse/ClickHouse/pull/12760) ([alesapin](https://github.com/alesapin)). +* Fix CAST(Nullable(String), Enum()). [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) ([Azat Khuzhin](https://github.com/azat)). +* Fix performance with large tuples, which are interpreted as functions in `IN` section. The case when user write `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). +* Fix memory tracking for input_format_parallel_parsing (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). +* fixes [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572) fix bloom filter index with const expression. [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). +* Fix SIGSEGV in StorageKafka when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). +* Add support for function `if` with `Array(UUID)` arguments. This fixes [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Better exception message in disk access storage. [#12625](https://github.com/ClickHouse/ClickHouse/pull/12625) ([alesapin](https://github.com/alesapin)). +* Fix lack of aliases with function `any`. [#12593](https://github.com/ClickHouse/ClickHouse/pull/12593) ([Anton Popov](https://github.com/CurtizJ)). +* Fix race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). +* Remove data for Distributed tables (blocks from async INSERTs) on DROP TABLE. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). +* Fix bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). +* Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). +* Fixing race condition in live view tables which could cause data duplication. [#12519](https://github.com/ClickHouse/ClickHouse/pull/12519) ([vzakaznikov](https://github.com/vzakaznikov)). +* Fixed performance issue, while reading from compact parts. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). +* Fix backwards compatibility in binary format of `AggregateFunction(avg, ...)` values. This fixes [#12342](https://github.com/ClickHouse/ClickHouse/issues/12342). [#12486](https://github.com/ClickHouse/ClickHouse/pull/12486) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix SETTINGS parse after FORMAT. [#12480](https://github.com/ClickHouse/ClickHouse/pull/12480) ([Azat Khuzhin](https://github.com/azat)). +* SystemLog: do not write to ordinary server log under mutex. This can lead to deadlock if `text_log` is enabled. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix overflow when very large LIMIT or OFFSET is specified. This fixes [#10470](https://github.com/ClickHouse/ClickHouse/issues/10470). This fixes [#11372](https://github.com/ClickHouse/ClickHouse/issues/11372). [#12427](https://github.com/ClickHouse/ClickHouse/pull/12427) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed possible segfault if StorageMerge. Closes [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). +* Reverts change introduced in [#11079](https://github.com/ClickHouse/ClickHouse/issues/11079) to resolve [#12098](https://github.com/ClickHouse/ClickHouse/issues/12098). [#12397](https://github.com/ClickHouse/ClickHouse/pull/12397) ([Mike](https://github.com/myrrc)). +* Additional check for arguments of bloom filter index. This fixes [#11408](https://github.com/ClickHouse/ClickHouse/issues/11408). [#12388](https://github.com/ClickHouse/ClickHouse/pull/12388) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Avoid exception when negative or floating point constant is used in WHERE condition for indexed tables. This fixes [#11905](https://github.com/ClickHouse/ClickHouse/issues/11905). [#12384](https://github.com/ClickHouse/ClickHouse/pull/12384) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow to CLEAR column even if there are depending DEFAULT expressions. This fixes [#12333](https://github.com/ClickHouse/ClickHouse/issues/12333). [#12378](https://github.com/ClickHouse/ClickHouse/pull/12378) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix TOTALS/ROLLUP/CUBE for aggregate functions with `-State` and `Nullable` arguments. This fixes [#12163](https://github.com/ClickHouse/ClickHouse/issues/12163). [#12376](https://github.com/ClickHouse/ClickHouse/pull/12376) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix error message and exit codes for `ALTER RENAME COLUMN` queries, when `RENAME` is not allowed. Fixes [#12301](https://github.com/ClickHouse/ClickHouse/issues/12301) and [#12303](https://github.com/ClickHouse/ClickHouse/issues/12303). [#12335](https://github.com/ClickHouse/ClickHouse/pull/12335) ([alesapin](https://github.com/alesapin)). +* Fix very rare race condition in ReplicatedMergeTreeQueue. [#12315](https://github.com/ClickHouse/ClickHouse/pull/12315) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* When using codec `Delta` or `DoubleDelta` with non fixed width types, exception with code `LOGICAL_ERROR` was returned instead of exception with code `BAD_ARGUMENTS` (we ensure that exceptions with code logical error never happen). This fixes [#12110](https://github.com/ClickHouse/ClickHouse/issues/12110). [#12308](https://github.com/ClickHouse/ClickHouse/pull/12308) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix order of columns in `WITH FILL` modifier. Previously order of columns of `ORDER BY` statement wasn't respected. [#12306](https://github.com/ClickHouse/ClickHouse/pull/12306) ([Anton Popov](https://github.com/CurtizJ)). +* Avoid "bad cast" exception when there is an expression that filters data by virtual columns (like `_table` in `Merge` tables) or by "index" columns in system tables such as filtering by database name when querying from `system.tables`, and this expression returns `Nullable` type. This fixes [#12166](https://github.com/ClickHouse/ClickHouse/issues/12166). [#12305](https://github.com/ClickHouse/ClickHouse/pull/12305) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix TTL after renaming column, on which depends TTL expression. [#12304](https://github.com/ClickHouse/ClickHouse/pull/12304) ([Anton Popov](https://github.com/CurtizJ)). +* kafka: fix SIGSEGV if there is an message with error in the middle of the batch. [#12302](https://github.com/ClickHouse/ClickHouse/pull/12302) ([Azat Khuzhin](https://github.com/azat)). +* Some threads might randomly hang for a few seconds during DNS cache updating. It's fixed. [#12296](https://github.com/ClickHouse/ClickHouse/pull/12296) ([tavplubix](https://github.com/tavplubix)). +* Fix typo in setting name. [#12292](https://github.com/ClickHouse/ClickHouse/pull/12292) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Show error after TrieDictionary failed to load. [#12290](https://github.com/ClickHouse/ClickHouse/pull/12290) ([Vitaly Baranov](https://github.com/vitlibar)). +* The function `arrayFill` worked incorrectly for empty arrays that may lead to crash. This fixes [#12263](https://github.com/ClickHouse/ClickHouse/issues/12263). [#12279](https://github.com/ClickHouse/ClickHouse/pull/12279) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Implement conversions to the common type for LowCardinality types. This allows to execute UNION ALL of tables with columns of LowCardinality and other columns. This fixes [#8212](https://github.com/ClickHouse/ClickHouse/issues/8212). This fixes [#4342](https://github.com/ClickHouse/ClickHouse/issues/4342). [#12275](https://github.com/ClickHouse/ClickHouse/pull/12275) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Not for changelog. Cherry-pick after [#12196](https://github.com/ClickHouse/ClickHouse/issues/12196). [#12271](https://github.com/ClickHouse/ClickHouse/pull/12271) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed behaviour on reaching redirect limit in request to S3 storage. [#12256](https://github.com/ClickHouse/ClickHouse/pull/12256) ([ianton-ru](https://github.com/ianton-ru)). +* Fixed the behaviour when during multiple sequential inserts in `StorageFile` header for some special types was written more than once. This fixed [#6155](https://github.com/ClickHouse/ClickHouse/issues/6155). [#12197](https://github.com/ClickHouse/ClickHouse/pull/12197) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed logical functions for UInt8 values when they are not equal to 0 or 1. [#12196](https://github.com/ClickHouse/ClickHouse/pull/12196) ([Alexander Kazakov](https://github.com/Akazz)). +* Cap max_memory_usage* limits to the process resident memory. [#12182](https://github.com/ClickHouse/ClickHouse/pull/12182) ([Azat Khuzhin](https://github.com/azat)). +* Fix dictGet arguments check during GROUP BY injective functions elimination. [#12179](https://github.com/ClickHouse/ClickHouse/pull/12179) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the behaviour when `SummingMergeTree` engine sums up columns from partition key. Added an exception in case of explicit definition of columns to sum which intersects with partition key columns. This fixes [#7867](https://github.com/ClickHouse/ClickHouse/issues/7867). [#12173](https://github.com/ClickHouse/ClickHouse/pull/12173) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Don't split the dictionary source's table name into schema and table name itself if ODBC connection doesn't support schema. [#12165](https://github.com/ClickHouse/ClickHouse/pull/12165) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix wrong logic in ALTER DELETE that leads to deleting of records when condition evaluates to NULL. This fixes [#9088](https://github.com/ClickHouse/ClickHouse/issues/9088). This closes [#12106](https://github.com/ClickHouse/ClickHouse/issues/12106). [#12153](https://github.com/ClickHouse/ClickHouse/pull/12153) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix transform of query to send to external DBMS (e.g. MySQL, ODBC) in presense of aliases. This fixes [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bad code in redundant ORDER BY optimization. The bug was introduced in [#10067](https://github.com/ClickHouse/ClickHouse/issues/10067). [#12148](https://github.com/ClickHouse/ClickHouse/pull/12148) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix potential overflow in integer division. This fixes [#12119](https://github.com/ClickHouse/ClickHouse/issues/12119). [#12140](https://github.com/ClickHouse/ClickHouse/pull/12140) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix potential infinite loop in `greatCircleDistance`, `geoDistance`. This fixes [#12117](https://github.com/ClickHouse/ClickHouse/issues/12117). [#12137](https://github.com/ClickHouse/ClickHouse/pull/12137) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Normalize "pid" file handling. In previous versions the server may refuse to start if it was killed without proper shutdown and if there is another process that has the same pid as previously runned server. Also pid file may be removed in unsuccessful server startup even if there is another server running. This fixes [#3501](https://github.com/ClickHouse/ClickHouse/issues/3501). [#12133](https://github.com/ClickHouse/ClickHouse/pull/12133) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bug which leads to incorrect table metadata in ZooKeepeer for ReplicatedVersionedCollapsingMergeTree tables. Fixes [#12093](https://github.com/ClickHouse/ClickHouse/issues/12093). [#12121](https://github.com/ClickHouse/ClickHouse/pull/12121) ([alesapin](https://github.com/alesapin)). +* Avoid "There is no query" exception for materialized views with joins or with subqueries attached to system logs (system.query_log, metric_log, etc) or to engine=Buffer underlying table. [#12120](https://github.com/ClickHouse/ClickHouse/pull/12120) ([filimonov](https://github.com/filimonov)). +* Fix handling dependency of table with ENGINE=Dictionary on dictionary. This fixes [#10994](https://github.com/ClickHouse/ClickHouse/issues/10994). This fixes [#10397](https://github.com/ClickHouse/ClickHouse/issues/10397). [#12116](https://github.com/ClickHouse/ClickHouse/pull/12116) ([Vitaly Baranov](https://github.com/vitlibar)). +* Format `Parquet` now properly works with `LowCardinality` and `LowCardinality(Nullable)` types. Fixes [#12086](https://github.com/ClickHouse/ClickHouse/issues/12086), [#8406](https://github.com/ClickHouse/ClickHouse/issues/8406). [#12108](https://github.com/ClickHouse/ClickHouse/pull/12108) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix performance for selects with `UNION` caused by wrong limit for the total number of threads. Fixes [#12030](https://github.com/ClickHouse/ClickHouse/issues/12030). [#12103](https://github.com/ClickHouse/ClickHouse/pull/12103) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix segfault with `-StateResample` combinators. [#12092](https://github.com/ClickHouse/ClickHouse/pull/12092) ([Anton Popov](https://github.com/CurtizJ)). +* Fix empty `result_rows` and `result_bytes` metrics in `system.quey_log` for selects. Fixes [#11595](https://github.com/ClickHouse/ClickHouse/issues/11595). [#12089](https://github.com/ClickHouse/ClickHouse/pull/12089) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix unnecessary limiting the number of threads for selects from `VIEW`. Fixes [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix SIGSEGV in StorageKafka on DROP TABLE. [#12075](https://github.com/ClickHouse/ClickHouse/pull/12075) ([Azat Khuzhin](https://github.com/azat)). +* Fix possible crash while using wrong type for `PREWHERE`. Fixes [#12053](https://github.com/ClickHouse/ClickHouse/issues/12053), [#12060](https://github.com/ClickHouse/ClickHouse/issues/12060). [#12060](https://github.com/ClickHouse/ClickHouse/pull/12060) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix error `Cannot capture column` for higher-order functions with `Tuple(LowCardinality)` argument. Fixes [#9766](https://github.com/ClickHouse/ClickHouse/issues/9766). [#12055](https://github.com/ClickHouse/ClickHouse/pull/12055) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix constraints check if constraint is a constant expression. This fixes [#11360](https://github.com/ClickHouse/ClickHouse/issues/11360). [#12042](https://github.com/ClickHouse/ClickHouse/pull/12042) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix wrong result and potential crash when invoking function `if` with arguments of type `FixedString` with different sizes. This fixes [#11362](https://github.com/ClickHouse/ClickHouse/issues/11362). [#12021](https://github.com/ClickHouse/ClickHouse/pull/12021) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Improvement + +* Allow to set JOIN kind and type in more standad way: `LEFT SEMI JOIN` instead of `SEMI LEFT JOIN`. For now both are correct. [#12520](https://github.com/ClickHouse/ClickHouse/pull/12520) ([Artem Zuikov](https://github.com/4ertus2)). +* lifetime_rows/lifetime_bytes for Buffer engine. [#12421](https://github.com/ClickHouse/ClickHouse/pull/12421) ([Azat Khuzhin](https://github.com/azat)). +* Write the detail exception message to the client instead of 'MySQL server has gone away'. [#12383](https://github.com/ClickHouse/ClickHouse/pull/12383) ([BohuTANG](https://github.com/BohuTANG)). +* This setting allows to chose charset for printing grids (either utf8 or ascii). [#12372](https://github.com/ClickHouse/ClickHouse/pull/12372) ([Sabyanin Maxim](https://github.com/s-mx)). +* 1. Support MySQL 'SELECT DATABASE()' [#9336](https://github.com/ClickHouse/ClickHouse/issues/9336) 2. Add MySQL replacement query integration test. [#12314](https://github.com/ClickHouse/ClickHouse/pull/12314) ([BohuTANG](https://github.com/BohuTANG)). +* Add `KILL QUERY [connection_id]` for the MySQL client/driver to cancel the long query, issue [#12038](https://github.com/ClickHouse/ClickHouse/issues/12038). [#12152](https://github.com/ClickHouse/ClickHouse/pull/12152) ([BohuTANG](https://github.com/BohuTANG)). +* Added support for `%g` (two digit ISO year) and `%G` (four digit ISO year) substitutions in `formatDateTime` function. [#12136](https://github.com/ClickHouse/ClickHouse/pull/12136) ([vivarum](https://github.com/vivarum)). +* Add 'type' column in system.disks. [#12115](https://github.com/ClickHouse/ClickHouse/pull/12115) ([ianton-ru](https://github.com/ianton-ru)). +* Improves `REVOKE` command: now it requires grant/admin option for only access which will be revoked. For example, to execute `REVOKE ALL ON *.* FROM user1` now it doesn't require to have full access rights granted with grant option. Added command `REVOKE ALL FROM user1` - it revokes all granted roles from `user1`. [#12083](https://github.com/ClickHouse/ClickHouse/pull/12083) ([Vitaly Baranov](https://github.com/vitlibar)). +* Implemented single part uploads for DiskS3. [#12026](https://github.com/ClickHouse/ClickHouse/pull/12026) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Add replica priority for load_balancing (for manual prioritization of the load balancing). [#11995](https://github.com/ClickHouse/ClickHouse/pull/11995) ([Azat Khuzhin](https://github.com/azat)). +* Improved performace of 'ORDER BY' and 'GROUP BY' by prefix of sorting key. [#11696](https://github.com/ClickHouse/ClickHouse/pull/11696) ([Anton Popov](https://github.com/CurtizJ)). + +#### Performance Improvement + +* Remove injective functions inside `uniq*()` if `set optimize_injective_functions_inside_uniq=1`. [#12337](https://github.com/ClickHouse/ClickHouse/pull/12337) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix "[#10574](https://github.com/ClickHouse/ClickHouse/issues/10574) Index not used for IN operator with literals", performance regression introduced around v19.3. [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). + +#### Build/Testing/Packaging Improvement + +* Added S3 HTTPS integration test. [#12412](https://github.com/ClickHouse/ClickHouse/pull/12412) ([Pavel Kovalenko](https://github.com/Jokser)). +* Log sanitizer trap messages from separate thread. This will prevent possible deadlock under thread sanitizer. [#12313](https://github.com/ClickHouse/ClickHouse/pull/12313) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now functional and stress tests will be able to run with old version of `clickhouse-test` script. [#12287](https://github.com/ClickHouse/ClickHouse/pull/12287) ([alesapin](https://github.com/alesapin)). +* Remove strange file creation during build in `orc`. [#12258](https://github.com/ClickHouse/ClickHouse/pull/12258) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Remove verbosity from the binary builds. [#12174](https://github.com/ClickHouse/ClickHouse/pull/12174) ([alesapin](https://github.com/alesapin)). +* Place common docker compose files to integration docker container. [#12168](https://github.com/ClickHouse/ClickHouse/pull/12168) ([Ilya Yatsishin](https://github.com/qoega)). +* Record additional detail on Dockerfile scan reports. [#12159](https://github.com/ClickHouse/ClickHouse/pull/12159) ([Ivan Blinkov](https://github.com/blinkov)). +* Fix warnings from CodeQL. `CodeQL` is another static analyzer that we will use along with `clang-tidy` and `PVS-Studio` that we use already. [#12138](https://github.com/ClickHouse/ClickHouse/pull/12138) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Missed `` is required for `std::atomic<>`. [#12134](https://github.com/ClickHouse/ClickHouse/pull/12134) ([Matwey V. Kornilov](https://github.com/matwey)). +* Minor CMake fixes for UNBUNDLED build. [#12131](https://github.com/ClickHouse/ClickHouse/pull/12131) ([Matwey V. Kornilov](https://github.com/matwey)). +* Added a showcase of the minimal Docker image without using any Linux distribution. [#12126](https://github.com/ClickHouse/ClickHouse/pull/12126) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Perform an upgrade of system packages in the `clickhouse-server` docker image. [#12124](https://github.com/ClickHouse/ClickHouse/pull/12124) ([Ivan Blinkov](https://github.com/blinkov)). +* Implement AST-based query fuzzing mode for clickhouse-client. See [this label](https://github.com/ClickHouse/ClickHouse/issues?q=label%3Afuzz+is%3Aissue) for the list of issues we recently found by fuzzing. Most of them were found by this tool, and a couple by SQLancer and `00746_sql_fuzzy.pl`. [#12111](https://github.com/ClickHouse/ClickHouse/pull/12111) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Add `UNBUNDLED` flag to `system.build_options` table. Move skip lists for `clickhouse-test` to clickhouse repo. [#12107](https://github.com/ClickHouse/ClickHouse/pull/12107) ([alesapin](https://github.com/alesapin)). +* Regular check by [Anchore Container Analysis](https://docs.anchore.com) security analysis tool that looks for [CVE](https://cve.mitre.org/) in `clickhouse-server` Docker image. Also confirms that `Dockerfile` is buildable. Runs daily on `master` and on pull-requests to `Dockerfile`. [#12102](https://github.com/ClickHouse/ClickHouse/pull/12102) ([Ivan Blinkov](https://github.com/blinkov)). +* Daily check by [GitHub CodeQL](https://securitylab.github.com/tools/codeql) security analysis tool that looks for [CWE](https://cwe.mitre.org/). [#12101](https://github.com/ClickHouse/ClickHouse/pull/12101) ([Ivan Blinkov](https://github.com/blinkov)). +* Install `ca-certificates` before the first `apt-get update` in Dockerfile. [#12095](https://github.com/ClickHouse/ClickHouse/pull/12095) ([Ivan Blinkov](https://github.com/blinkov)). +* Add new type of tests based on Testflows framework. [#12090](https://github.com/ClickHouse/ClickHouse/pull/12090) ([vzakaznikov](https://github.com/vzakaznikov)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Backport [#12700](https://github.com/ClickHouse/ClickHouse/issues/12700) to 20.6: Fix performance with large tuples'. [#13187](https://github.com/ClickHouse/ClickHouse/pull/13187) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#13075](https://github.com/ClickHouse/ClickHouse/issues/13075) to 20.6: Fix 11327'. [#13184](https://github.com/ClickHouse/ClickHouse/pull/13184) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#13081](https://github.com/ClickHouse/ClickHouse/issues/13081) to 20.6: Fix wrong index analysis with functions'. [#13146](https://github.com/ClickHouse/ClickHouse/pull/13146) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#13009](https://github.com/ClickHouse/ClickHouse/issues/13009) to 20.6: Fix 12623'. [#13051](https://github.com/ClickHouse/ClickHouse/pull/13051) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12989](https://github.com/ClickHouse/ClickHouse/issues/12989) to 20.6: Fix `Block structure mismatch` error for queries with UNION and JOIN'. [#13049](https://github.com/ClickHouse/ClickHouse/pull/13049) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12982](https://github.com/ClickHouse/ClickHouse/issues/12982) to 20.6: Merging [#12548](https://github.com/ClickHouse/ClickHouse/issues/12548) - Correction to `merge_with_ttl_timeout` logic by @excitoon'. [#13031](https://github.com/ClickHouse/ClickHouse/pull/13031) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12760](https://github.com/ClickHouse/ClickHouse/issues/12760) to 20.6: Sticking mutations bug'. [#13011](https://github.com/ClickHouse/ClickHouse/pull/13011) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12664](https://github.com/ClickHouse/ClickHouse/issues/12664) to 20.6: Fix move_functions_out_of_any optimisation with lambda'. [#12994](https://github.com/ClickHouse/ClickHouse/pull/12994) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12658](https://github.com/ClickHouse/ClickHouse/issues/12658) to 20.6: Fix SIGSEGV in StorageKafka when broker is unavailable'. [#12971](https://github.com/ClickHouse/ClickHouse/pull/12971) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12663](https://github.com/ClickHouse/ClickHouse/issues/12663) to 20.6: ISSUES-12293 allow push predicate when subquery contains with clause'. [#12868](https://github.com/ClickHouse/ClickHouse/pull/12868) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12672](https://github.com/ClickHouse/ClickHouse/issues/12672) to 20.6: Fix memory tracking for input_format_parallel_parsing'. [#12864](https://github.com/ClickHouse/ClickHouse/pull/12864) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12492](https://github.com/ClickHouse/ClickHouse/issues/12492) to 20.6: Make read buffer size lower, while reading from compact parts'. [#12862](https://github.com/ClickHouse/ClickHouse/pull/12862) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12659](https://github.com/ClickHouse/ClickHouse/issues/12659) to 20.6: ISSUES-10572 fix bloom filter with const column'. [#12858](https://github.com/ClickHouse/ClickHouse/pull/12858) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12480](https://github.com/ClickHouse/ClickHouse/issues/12480) to 20.6: [RFC] Fix SETTINGS parse after FORMAT'. [#12803](https://github.com/ClickHouse/ClickHouse/pull/12803) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12458](https://github.com/ClickHouse/ClickHouse/issues/12458) to 20.6: Fix crash in 'JOIN dict ON expr(dict_key) = k''. [#12725](https://github.com/ClickHouse/ClickHouse/pull/12725) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12302](https://github.com/ClickHouse/ClickHouse/issues/12302) to 20.6: kafka: fix SIGSEGV if there is a message with error in the middle of the batch'. [#12724](https://github.com/ClickHouse/ClickHouse/pull/12724) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12382](https://github.com/ClickHouse/ClickHouse/issues/12382) to 20.6: Better errors for CLEAR/DROP columns (possibly in partitions)'. [#12723](https://github.com/ClickHouse/ClickHouse/pull/12723) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12519](https://github.com/ClickHouse/ClickHouse/issues/12519) to 20.6: Fixing race condition in live view tables which could cause data duplication and live view tests'. [#12722](https://github.com/ClickHouse/ClickHouse/pull/12722) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12545](https://github.com/ClickHouse/ClickHouse/issues/12545) to 20.6: Fix loading parts without checksums'. [#12721](https://github.com/ClickHouse/ClickHouse/pull/12721) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12624](https://github.com/ClickHouse/ClickHouse/issues/12624) to 20.6: Fix error message about granularity'. [#12720](https://github.com/ClickHouse/ClickHouse/pull/12720) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12618](https://github.com/ClickHouse/ClickHouse/issues/12618) to 20.6: Fix external sort pipeline stuck'. [#12699](https://github.com/ClickHouse/ClickHouse/pull/12699) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12633](https://github.com/ClickHouse/ClickHouse/issues/12633) to 20.6: Fix "There is no supertype" error on ALTER UPDATE [#7306](https://github.com/ClickHouse/ClickHouse/issues/7306)'. [#12698](https://github.com/ClickHouse/ClickHouse/pull/12698) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12648](https://github.com/ClickHouse/ClickHouse/issues/12648) to 20.6: Add support for function if with Array(UUID) arguments'. [#12697](https://github.com/ClickHouse/ClickHouse/pull/12697) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12625](https://github.com/ClickHouse/ClickHouse/issues/12625) to 20.6: Better exception during directories creation'. [#12696](https://github.com/ClickHouse/ClickHouse/pull/12696) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* NO CL ENTRY: 'Backport [#12613](https://github.com/ClickHouse/ClickHouse/issues/12613) to 20.6: Remove sort description from streams'. [#12695](https://github.com/ClickHouse/ClickHouse/pull/12695) ([robot-clickhouse](https://github.com/robot-clickhouse)). + +#### NO CL CATEGORY + +* ... [#12431](https://github.com/ClickHouse/ClickHouse/pull/12431) ([Tom Bombadil](https://github.com/ithangzhou)). +* * Not for changelog. [#12265](https://github.com/ClickHouse/ClickHouse/pull/12265) ([Alexander Kuzmenkov](https://github.com/akuzm)). + + + ## ClickHouse release 20.5 ### ClickHouse release v20.5.2.7-stable 2020-07-02 From c32fddbb6d0e47e203f758c048f5bf42fbad8e4e Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 6 Aug 2020 18:39:09 +0300 Subject: [PATCH 132/374] Do not fuzz CREATE queries We don't do it anyway, the first unmodified query succeeds and all others fail. Just make it official. --- programs/client/Client.cpp | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 440d080637f..bf51250210e 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -57,7 +57,7 @@ #include #include #include -#include +#include #include #include #include @@ -67,6 +67,7 @@ #include #include #include +#include #include #include #include @@ -1040,9 +1041,21 @@ private: full_query = text.substr(this_query_begin - text.data(), begin - text.data()); + // Don't repeat inserts, the tables grow too big. Also don't repeat + // creates because first we run the unmodified query, it will succeed, + // and the subsequent queries will fail. When we run out of fuzzer + // errors, it may be interesting to add fuzzing of create queries that + // wraps columns into LowCardinality or Nullable. Also there are other + // kinds of create queries such as CREATE DICTIONARY, we could fuzz + // them as well. + int this_query_runs = query_fuzzer_runs; + if (as_insert + || orig_ast->as()) + { + this_query_runs = 1; + } + ASTPtr fuzz_base = orig_ast; - // Don't repeat inserts, the tables grow too big. - const int this_query_runs = as_insert ? 1 : query_fuzzer_runs; for (int fuzz_step = 0; fuzz_step < this_query_runs; fuzz_step++) { fprintf(stderr, "fuzzing step %d out of %d for query at pos %zd\n", From b9fdc49e1a285085dd3af31ad7d62b97d5018783 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 6 Aug 2020 22:44:42 +0300 Subject: [PATCH 133/374] try fix synchronization check --- src/Databases/MySQL/MaterializeMetadata.cpp | 7 ++++++- src/Databases/MySQL/MaterializeMetadata.h | 7 +++++++ src/Interpreters/InterpreterExternalDDLQuery.cpp | 2 +- src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 6 +++--- src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp | 3 +-- 5 files changed, 18 insertions(+), 7 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 0b470dda903..80484c130d3 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -1,7 +1,10 @@ +#include + +#if USE_MYSQL + #include #include #include -#include #include #include #include @@ -206,3 +209,5 @@ MaterializeMetadata::MaterializeMetadata( } } + +#endif diff --git a/src/Databases/MySQL/MaterializeMetadata.h b/src/Databases/MySQL/MaterializeMetadata.h index c377a99a27c..045dab5b84c 100644 --- a/src/Databases/MySQL/MaterializeMetadata.h +++ b/src/Databases/MySQL/MaterializeMetadata.h @@ -1,5 +1,11 @@ #pragma once +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#if USE_MYSQL + #include #include #include @@ -42,3 +48,4 @@ struct MaterializeMetadata } +#endif diff --git a/src/Interpreters/InterpreterExternalDDLQuery.cpp b/src/Interpreters/InterpreterExternalDDLQuery.cpp index 89b8e598ac9..4a93c0fa753 100644 --- a/src/Interpreters/InterpreterExternalDDLQuery.cpp +++ b/src/Interpreters/InterpreterExternalDDLQuery.cpp @@ -3,6 +3,7 @@ #endif #include +#include #include #include @@ -11,7 +12,6 @@ #include #ifdef USE_MYSQL -# include # include # include # include diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index ca39a8ae4fc..95bd8c44d30 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -165,7 +165,7 @@ static inline std::tuplechanges.count("primary_key")) primary_keys->arguments->children.emplace_back(std::make_shared(declare_column->name)); - if (options->changes.contains("auto_increment")) + if (options->changes.count("auto_increment")) increment_columns.emplace(declare_column->name); } } @@ -272,10 +272,10 @@ static ASTPtr getOrderByPolicy( for (const auto & [name, type] : names_and_types) { - if (order_by_columns_set.contains(name)) + if (order_by_columns_set.count(name)) continue; - if (increment_columns.contains(name)) + if (increment_columns.count(name)) { increment_keys.emplace_back(name); order_by_columns_set.emplace(name); diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index 9fb71cf3a14..3782d20bd3d 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -16,12 +16,11 @@ using namespace DB; -static inline ASTPtr tryRewrittenCreateQuery(const String & query, const Context & context_) +static inline ASTPtr tryRewrittenCreateQuery(const String & query, const Context & context) { ParserExternalDDLQuery external_ddl_parser; ASTPtr ast = parseQuery(external_ddl_parser, query, 0, 0); - Context context = context_; return MySQLInterpreter::InterpreterCreateImpl::getRewrittenQuery( *ast->as()->external_ddl->as(), context, "test_database", "test_database"); From bd67ae6052f6cd9d3d0abdf43d810255e3130aff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Aug 2020 03:00:35 +0300 Subject: [PATCH 134/374] Fix tests depending on floating point associativity --- .../01300_group_by_other_keys.reference | 96 +++++++++---------- .../0_stateless/01300_group_by_other_keys.sql | 20 ++-- ...01300_group_by_other_keys_having.reference | 24 ++--- .../01300_group_by_other_keys_having.sql | 8 +- 4 files changed, 74 insertions(+), 74 deletions(-) diff --git a/tests/queries/0_stateless/01300_group_by_other_keys.reference b/tests/queries/0_stateless/01300_group_by_other_keys.reference index 7f0661ec4fe..bd2372dca93 100644 --- a/tests/queries/0_stateless/01300_group_by_other_keys.reference +++ b/tests/queries/0_stateless/01300_group_by_other_keys.reference @@ -1,27 +1,27 @@ -6931467.646716369 -6931468.33986355 -6931469.0330107305 -6931469.726157911 -6931470.419305092 -6931471.112452272 -3465734.169931768 -3465734.8630789486 -3465735.5562261306 -3465736.24937331 -3465736.94252049 -3465735.209652544 -3465735.209652544 -3465735.5562261483 -3465735.9027997246 -3465735.902799725 -3465734.516505364 -3465735.209652544 -3465735.209652544 -3465735.9027997246 -3465735.902799725 -3465736.595946905 -3465735.2096525617 -3465735.9027997428 +6931467.646716 +6931468.339864 +6931469.033011 +6931469.726158 +6931470.419305 +6931471.112452 +3465734.169932 +3465734.863079 +3465735.556226 +3465736.249373 +3465736.94252 +3465735.209653 +3465735.209653 +3465735.556226 +3465735.9028 +3465735.9028 +3465734.516505 +3465735.209653 +3465735.209653 +3465735.9028 +3465735.9028 +3465736.595947 +3465735.209653 +3465735.9028 SELECT max(log(2) * number) AS k FROM numbers(10000000) GROUP BY @@ -51,30 +51,30 @@ GROUP BY (number % 2) % 3, number % 2 ORDER BY k ASC -6931467.646716369 -6931468.33986355 -6931469.0330107305 -6931469.726157911 -6931470.419305092 -6931471.112452272 -3465734.169931768 -3465734.8630789486 -3465735.5562261306 -3465736.24937331 -3465736.94252049 -3465735.209652544 -3465735.209652544 -3465735.5562261483 -3465735.9027997246 -3465735.902799725 -3465734.516505364 -3465735.209652544 -3465735.209652544 -3465735.9027997246 -3465735.902799725 -3465736.595946905 -3465735.2096525617 -3465735.9027997428 +6931467.646716 +6931468.339864 +6931469.033011 +6931469.726158 +6931470.419305 +6931471.112452 +3465734.169932 +3465734.863079 +3465735.556226 +3465736.249373 +3465736.94252 +3465735.209653 +3465735.209653 +3465735.556226 +3465735.9028 +3465735.9028 +3465734.516505 +3465735.209653 +3465735.209653 +3465735.9028 +3465735.9028 +3465736.595947 +3465735.209653 +3465735.9028 SELECT max(log(2) * number) AS k FROM numbers(10000000) GROUP BY diff --git a/tests/queries/0_stateless/01300_group_by_other_keys.sql b/tests/queries/0_stateless/01300_group_by_other_keys.sql index d28f3707f7a..4dcd9cf1efe 100644 --- a/tests/queries/0_stateless/01300_group_by_other_keys.sql +++ b/tests/queries/0_stateless/01300_group_by_other_keys.sql @@ -1,11 +1,11 @@ set optimize_group_by_function_keys = 1; set enable_debug_queries = 1; -SELECT max(log(2) * number) AS k FROM numbers(10000000) GROUP BY number % 2, number % 3, (number % 2 + number % 3) % 2 ORDER BY k; -SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) ORDER BY k; -SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3 ORDER BY k; -SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3, number % 2 ORDER BY k; -SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY (number % 2) % 3, number % 2 ORDER BY k; +SELECT round(max(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY number % 2, number % 3, (number % 2 + number % 3) % 2 ORDER BY k; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) ORDER BY k; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3 ORDER BY k; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3, number % 2 ORDER BY k; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY (number % 2) % 3, number % 2 ORDER BY k; analyze SELECT max(log(2) * number) AS k FROM numbers(10000000) GROUP BY number % 2, number % 3, (number % 2 + number % 3) % 2 ORDER BY k; @@ -16,11 +16,11 @@ analyze SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY (number set optimize_group_by_function_keys = 0; -SELECT max(log(2) * number) AS k FROM numbers(10000000) GROUP BY number % 2, number % 3, (number % 2 + number % 3) % 2 ORDER BY k; -SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) ORDER BY k; -SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3 ORDER BY k; -SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3, number % 2 ORDER BY k; -SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY (number % 2) % 3, number % 2 ORDER BY k; +SELECT round(max(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY number % 2, number % 3, (number % 2 + number % 3) % 2 ORDER BY k; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) ORDER BY k; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3 ORDER BY k; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3, number % 2 ORDER BY k; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY (number % 2) % 3, number % 2 ORDER BY k; analyze SELECT max(log(2) * number) AS k FROM numbers(10000000) GROUP BY number % 2, number % 3, (number % 2 + number % 3) % 2 ORDER BY k; analyze SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) ORDER BY k; diff --git a/tests/queries/0_stateless/01300_group_by_other_keys_having.reference b/tests/queries/0_stateless/01300_group_by_other_keys_having.reference index 67262193df5..0bec0ebdf9b 100644 --- a/tests/queries/0_stateless/01300_group_by_other_keys_having.reference +++ b/tests/queries/0_stateless/01300_group_by_other_keys_having.reference @@ -1,9 +1,9 @@ -3465735.9027997246 -3465735.902799725 -3465736.595946905 -3465734.169931768 -3465734.8630789486 -3465735.5562261306 +3465735.9028 +3465735.9028 +3465736.595947 +3465734.169932 +3465734.863079 +3465735.556226 0 1 4 @@ -24,12 +24,12 @@ FROM numbers(10000000) WHERE ((number % 5) * (number % 5)) < 5 GROUP BY number % 5 ORDER BY k ASC -3465735.9027997246 -3465735.902799725 -3465736.595946905 -3465734.169931768 -3465734.8630789486 -3465735.5562261306 +3465735.9028 +3465735.9028 +3465736.595947 +3465734.169932 +3465734.863079 +3465735.556226 0 1 4 diff --git a/tests/queries/0_stateless/01300_group_by_other_keys_having.sql b/tests/queries/0_stateless/01300_group_by_other_keys_having.sql index b359c074c44..d97d80b2f12 100644 --- a/tests/queries/0_stateless/01300_group_by_other_keys_having.sql +++ b/tests/queries/0_stateless/01300_group_by_other_keys_having.sql @@ -1,8 +1,8 @@ set optimize_group_by_function_keys = 1; set enable_debug_queries = 1; -SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3, number % 2 HAVING avg(log(2) * number) > 3465735.3 ORDER BY k; -SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3, number % 2 HAVING avg(log(2) * number) > 3465735.3 ORDER BY k; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k; SELECT (number % 5) * (number % 5) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k; @@ -12,8 +12,8 @@ analyze SELECT (number % 5) * (number % 5) AS k FROM numbers(10000000) GROUP BY set optimize_group_by_function_keys = 0; -SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3, number % 2 HAVING avg(log(2) * number) > 3465735.3 ORDER BY k; -SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3, number % 2 HAVING avg(log(2) * number) > 3465735.3 ORDER BY k; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k; SELECT (number % 5) * (number % 5) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k; analyze SELECT avg(log(2) * number) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3, number % 2 HAVING avg(log(2) * number) > 3465735.3 ORDER BY k; From 733bd0ecec8266a4cc5ec095430b184294402f1c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Aug 2020 03:29:52 +0300 Subject: [PATCH 135/374] Fix error --- src/AggregateFunctions/IAggregateFunction.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index c3968db189f..568303ff246 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -351,7 +351,7 @@ public: /// If the function is complex or too large, use more generic algorithm. - if (func.allocatesMemoryInArena() || sizeof(Data) > 16) + if (func.allocatesMemoryInArena() || func.sizeOfData() > 16 || func.sizeOfData() != sizeof(Data)) { IAggregateFunctionHelper::addBatchLookupTable8(batch_size, map, place_offset, init, key, columns, arena); return; From cf618c1e2333d31057f0f2644eae32d24bdd787b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Aug 2020 03:33:17 +0300 Subject: [PATCH 136/374] Remove false statements --- src/AggregateFunctions/IAggregateFunction.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 568303ff246..fd3999adb6d 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -78,7 +78,7 @@ public: /// Get `sizeof` of structure with data. virtual size_t sizeOfData() const = 0; - /// How the data structure should be aligned. NOTE: Currently not used (structures with aggregation state are put without alignment). + /// How the data structure should be aligned. virtual size_t alignOfData() const = 0; /** Adds a value into aggregation data on which place points to. @@ -332,7 +332,6 @@ public: return sizeof(Data); } - /// NOTE: Currently not used (structures with aggregation state are put without alignment). size_t alignOfData() const override { return alignof(Data); From 476da6c1fe2977dc2fc5b6e7c3ae89148be818c6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Aug 2020 03:35:13 +0300 Subject: [PATCH 137/374] Slightly better --- src/AggregateFunctions/IAggregateFunction.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index fd3999adb6d..899c7d3e8b8 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -350,7 +350,7 @@ public: /// If the function is complex or too large, use more generic algorithm. - if (func.allocatesMemoryInArena() || func.sizeOfData() > 16 || func.sizeOfData() != sizeof(Data)) + if (func.allocatesMemoryInArena() || sizeof(Data) > 16 || func.sizeOfData() != sizeof(Data)) { IAggregateFunctionHelper::addBatchLookupTable8(batch_size, map, place_offset, init, key, columns, arena); return; From d1ed1fb83a577125028d784995e109fa8d167924 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Aug 2020 04:04:37 +0300 Subject: [PATCH 138/374] Fix error --- base/common/StringRef.h | 7 ++++++- src/Functions/URL/netloc.h | 19 ++++++++++++------- .../0_stateless/01434_netloc_fuzz.reference | 1 + .../queries/0_stateless/01434_netloc_fuzz.sql | 1 + 4 files changed, 20 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/01434_netloc_fuzz.reference create mode 100644 tests/queries/0_stateless/01434_netloc_fuzz.sql diff --git a/base/common/StringRef.h b/base/common/StringRef.h index df659f50123..2247f0de2ed 100644 --- a/base/common/StringRef.h +++ b/base/common/StringRef.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -27,7 +28,11 @@ struct StringRef size_t size = 0; template > - constexpr StringRef(const CharT * data_, size_t size_) : data(reinterpret_cast(data_)), size(size_) {} + constexpr StringRef(const CharT * data_, size_t size_) : data(reinterpret_cast(data_)), size(size_) + { + /// Sanity check for overflowed values. + assert(size < 0x8000000000000000ULL); + } StringRef(const std::string & s) : data(s.data()), size(s.size()) {} constexpr explicit StringRef(const std::string_view & s) : data(s.data()), size(s.size()) {} diff --git a/src/Functions/URL/netloc.h b/src/Functions/URL/netloc.h index 443ef7f9003..ac1e57a884a 100644 --- a/src/Functions/URL/netloc.h +++ b/src/Functions/URL/netloc.h @@ -20,7 +20,8 @@ struct ExtractNetloc Pos pos = data; Pos end = data + size; - if (*pos == '/' && *(pos + 1) == '/') + /// Skip scheme. + if (pos + 2 < end && pos[0] == '/' && pos[1] == '/') { pos += 2; } @@ -55,16 +56,18 @@ struct ExtractNetloc case '&': return StringRef{}; default: - goto exloop; + pos = scheme_end; /// exit from the loop } } } -exloop: if ((scheme_end - pos) > 2 && *pos == ':' && *(pos + 1) == '/' && *(pos + 2) == '/') - pos += 3; - else - pos = data; + if (pos + 2 < scheme_end && pos[0] == ':' && pos[1] == '/' && pos[2] == '/') + pos += 3; + else + pos = data; } + /// Now pos points to the first byte after scheme (if there is). + bool has_identification = false; Pos question_mark_pos = end; Pos slash_pos = end; @@ -106,7 +109,9 @@ exloop: if ((scheme_end - pos) > 2 && *pos == ':' && *(pos + 1) == '/' && *(pos case ';': case '=': case '&': - return StringRef(start_of_host, std::min(std::min(pos - 1, question_mark_pos), slash_pos) - start_of_host); + return pos > start_of_host + ? StringRef(start_of_host, std::min(std::min(pos - 1, question_mark_pos), slash_pos) - start_of_host) + : StringRef{}; } } diff --git a/tests/queries/0_stateless/01434_netloc_fuzz.reference b/tests/queries/0_stateless/01434_netloc_fuzz.reference new file mode 100644 index 00000000000..8b137891791 --- /dev/null +++ b/tests/queries/0_stateless/01434_netloc_fuzz.reference @@ -0,0 +1 @@ + diff --git a/tests/queries/0_stateless/01434_netloc_fuzz.sql b/tests/queries/0_stateless/01434_netloc_fuzz.sql new file mode 100644 index 00000000000..a409add313f --- /dev/null +++ b/tests/queries/0_stateless/01434_netloc_fuzz.sql @@ -0,0 +1 @@ +SELECT netloc('<\'[%UzO'); From 1f2d0dce3e0da7af47fc548a0bd3ec9b9be03e76 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Aug 2020 04:07:51 +0300 Subject: [PATCH 139/374] Remove useless header file --- src/Functions/URL/netloc.cpp | 125 +++++++++++++++++++++++++++++++- src/Functions/URL/netloc.h | 134 ----------------------------------- 2 files changed, 124 insertions(+), 135 deletions(-) delete mode 100644 src/Functions/URL/netloc.h diff --git a/src/Functions/URL/netloc.cpp b/src/Functions/URL/netloc.cpp index d8858c3364a..ea45504fa18 100644 --- a/src/Functions/URL/netloc.cpp +++ b/src/Functions/URL/netloc.cpp @@ -1,10 +1,133 @@ +#include #include #include -#include "netloc.h" +#include + namespace DB { +struct ExtractNetloc +{ + /// We use the same as domain function + static size_t getReserveLengthForElement() { return 15; } + + static inline StringRef getNetworkLocation(const char * data, size_t size) + { + Pos pos = data; + Pos end = data + size; + + /// Skip scheme. + if (pos + 2 < end && pos[0] == '/' && pos[1] == '/') + { + pos += 2; + } + else + { + Pos scheme_end = data + std::min(size, 16UL); + for (++pos; pos < scheme_end; ++pos) + { + if (!isAlphaNumericASCII(*pos)) + { + switch (*pos) + { + case '.': + case '-': + case '+': + break; + case ' ': /// restricted symbols + case '\t': + case '<': + case '>': + case '%': + case '{': + case '}': + case '|': + case '\\': + case '^': + case '~': + case '[': + case ']': + case ';': + case '=': + case '&': + return StringRef{}; + default: + pos = scheme_end; /// exit from the loop + } + } + } + if (pos + 2 < scheme_end && pos[0] == ':' && pos[1] == '/' && pos[2] == '/') + pos += 3; + else + pos = data; + } + + /// Now pos points to the first byte after scheme (if there is). + + bool has_identification = false; + Pos question_mark_pos = end; + Pos slash_pos = end; + auto start_of_host = pos; + for (; pos < end; ++pos) + { + switch (*pos) + { + case '/': + if (has_identification) + return StringRef(start_of_host, pos - start_of_host); + else + slash_pos = pos; + break; + case '?': + if (has_identification) + return StringRef(start_of_host, pos - start_of_host); + else + question_mark_pos = pos; + break; + case '#': + return StringRef(start_of_host, pos - start_of_host); + case '@': /// foo:bar@example.ru + has_identification = true; + break; + case ' ': /// restricted symbols in whole URL + case '\t': + case '<': + case '>': + case '%': + case '{': + case '}': + case '|': + case '\\': + case '^': + case '~': + case '[': + case ']': + case ';': + case '=': + case '&': + return pos > start_of_host + ? StringRef(start_of_host, std::min(std::min(pos - 1, question_mark_pos), slash_pos) - start_of_host) + : StringRef{}; + } + } + + if (has_identification) + return StringRef(start_of_host, pos - start_of_host); + else + return StringRef(start_of_host, std::min(std::min(pos, question_mark_pos), slash_pos) - start_of_host); + } + + static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) + { + StringRef host = getNetworkLocation(data, size); + + res_data = host.data; + res_size = host.size; + } +}; + + struct NameNetloc { static constexpr auto name = "netloc"; }; using FunctionNetloc = FunctionStringToString, NameNetloc>; diff --git a/src/Functions/URL/netloc.h b/src/Functions/URL/netloc.h deleted file mode 100644 index ac1e57a884a..00000000000 --- a/src/Functions/URL/netloc.h +++ /dev/null @@ -1,134 +0,0 @@ -#pragma once - -#include "FunctionsURL.h" -#include -#include "protocol.h" -#include -#include - - -namespace DB -{ - -struct ExtractNetloc -{ - /// We use the same as domain function - static size_t getReserveLengthForElement() { return 15; } - - static inline StringRef getNetworkLocation(const char * data, size_t size) - { - Pos pos = data; - Pos end = data + size; - - /// Skip scheme. - if (pos + 2 < end && pos[0] == '/' && pos[1] == '/') - { - pos += 2; - } - else - { - Pos scheme_end = data + std::min(size, 16UL); - for (++pos; pos < scheme_end; ++pos) - { - if (!isAlphaNumericASCII(*pos)) - { - switch (*pos) - { - case '.': - case '-': - case '+': - break; - case ' ': /// restricted symbols - case '\t': - case '<': - case '>': - case '%': - case '{': - case '}': - case '|': - case '\\': - case '^': - case '~': - case '[': - case ']': - case ';': - case '=': - case '&': - return StringRef{}; - default: - pos = scheme_end; /// exit from the loop - } - } - } - if (pos + 2 < scheme_end && pos[0] == ':' && pos[1] == '/' && pos[2] == '/') - pos += 3; - else - pos = data; - } - - /// Now pos points to the first byte after scheme (if there is). - - bool has_identification = false; - Pos question_mark_pos = end; - Pos slash_pos = end; - auto start_of_host = pos; - for (; pos < end; ++pos) - { - switch (*pos) - { - case '/': - if (has_identification) - return StringRef(start_of_host, pos - start_of_host); - else - slash_pos = pos; - break; - case '?': - if (has_identification) - return StringRef(start_of_host, pos - start_of_host); - else - question_mark_pos = pos; - break; - case '#': - return StringRef(start_of_host, pos - start_of_host); - case '@': /// foo:bar@example.ru - has_identification = true; - break; - case ' ': /// restricted symbols in whole URL - case '\t': - case '<': - case '>': - case '%': - case '{': - case '}': - case '|': - case '\\': - case '^': - case '~': - case '[': - case ']': - case ';': - case '=': - case '&': - return pos > start_of_host - ? StringRef(start_of_host, std::min(std::min(pos - 1, question_mark_pos), slash_pos) - start_of_host) - : StringRef{}; - } - } - - if (has_identification) - return StringRef(start_of_host, pos - start_of_host); - else - return StringRef(start_of_host, std::min(std::min(pos, question_mark_pos), slash_pos) - start_of_host); - } - - static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) - { - StringRef host = getNetworkLocation(data, size); - - res_data = host.data; - res_size = host.size; - } -}; - -} - From e4962256419f70019b4e51b1614637896d5b3d79 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 6 Aug 2020 19:56:49 +0800 Subject: [PATCH 140/374] ISSUES-4006 try fix build failure & review comment --- src/Databases/DatabaseAtomic.cpp | 4 +- src/Databases/DatabaseAtomic.h | 2 +- src/Databases/DatabaseLazy.cpp | 3 +- src/Databases/DatabaseLazy.h | 2 +- src/Databases/DatabaseOrdinary.cpp | 2 +- src/Databases/DatabaseOrdinary.h | 3 +- src/Databases/IDatabase.h | 2 +- .../MySQL/DatabaseConnectionMySQL.cpp | 2 +- src/Databases/MySQL/DatabaseConnectionMySQL.h | 2 +- .../MySQL/DatabaseMaterializeMySQL.cpp | 80 ++++++++++--------- .../MySQL/DatabaseMaterializeMySQL.h | 8 +- .../MySQL/DatabaseMaterializeTablesIterator.h | 10 ++- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.h | 6 ++ .../MySQL/tests/gtest_create_rewritten.cpp | 1 - src/Interpreters/loadMetadata.cpp | 1 + src/Storages/StorageMaterializeMySQL.cpp | 10 ++- src/Storages/StorageMaterializeMySQL.h | 6 +- 18 files changed, 84 insertions(+), 62 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 1223f9a1d56..227f4b661db 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -351,13 +351,13 @@ UUID DatabaseAtomic::tryGetTableUUID(const String & table_name) const return UUIDHelpers::Nil; } -void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag) +void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) { /// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken if (has_force_restore_data_flag) Poco::File(path_to_table_symlinks).remove(true); - DatabaseOrdinary::loadStoredObjects(context, has_force_restore_data_flag); + DatabaseOrdinary::loadStoredObjects(context, has_force_restore_data_flag, force_attach); if (has_force_restore_data_flag) { diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index f809fcefef3..02c922f8b91 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -48,7 +48,7 @@ public: DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; - void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override; + void loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) override; /// Atomic database cannot be detached if there is detached table which still in use void assertCanBeDetached(bool cleenup); diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index a4be82690e4..0119f17f843 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -36,7 +36,8 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, void DatabaseLazy::loadStoredObjects( Context & context, - bool /* has_force_restore_data_flag */) + bool /* has_force_restore_data_flag */, + bool /*force_attach*/) { iterateMetadataFiles(context, [this](const String & file_name) { diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index 3d3e5a8aed6..13c14863efb 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -24,7 +24,7 @@ public: void loadStoredObjects( Context & context, - bool has_force_restore_data_flag) override; + bool has_force_restore_data_flag, bool force_attach) override; void createTable( const Context & context, diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 45a7c1d2c66..7db51e826a3 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -105,7 +105,7 @@ DatabaseOrdinary::DatabaseOrdinary( { } -void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_restore_data_flag) +void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool /*force_attach*/) { /** Tables load faster if they are loaded in sorted (by name) order. * Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order, diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index a9e53edfe28..07783313413 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -21,7 +21,8 @@ public: void loadStoredObjects( Context & context, - bool has_force_restore_data_flag) override; + bool has_force_restore_data_flag, + bool force_attach) override; void alterTable( const Context & context, diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 34bcb93c5d8..d82755a7bc8 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -149,7 +149,7 @@ public: /// Load a set of existing tables. /// You can call only once, right after the object is created. - virtual void loadStoredObjects(Context & /*context*/, bool /*has_force_restore_data_flag*/) {} + virtual void loadStoredObjects(Context & /*context*/, bool /*has_force_restore_data_flag*/, bool /*force_attach*/ = false) {} /// Check the existence of the table. virtual bool isTableExist(const String & name, const Context & context) const = 0; diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 48c52773216..0d944e215a0 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -419,7 +419,7 @@ String DatabaseConnectionMySQL::getMetadataPath() const return metadata_path; } -void DatabaseConnectionMySQL::loadStoredObjects(Context &, bool) +void DatabaseConnectionMySQL::loadStoredObjects(Context &, bool, bool /*force_attach*/) { std::lock_guard lock{mutex}; diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.h b/src/Databases/MySQL/DatabaseConnectionMySQL.h index 49c2c65bfa4..c4fb3d5f90c 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.h +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.h @@ -50,7 +50,7 @@ public: void createTable(const Context &, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override; - void loadStoredObjects(Context &, bool) override; + void loadStoredObjects(Context &, bool, bool force_attach) override; StoragePtr detachTable(const String & table_name) override; diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 031384bef2c..1d3419a218c 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -33,13 +33,7 @@ DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( { } -void DatabaseMaterializeMySQL::setException(const std::exception_ptr & exception_) -{ - std::unique_lock lock(mutex); - exception = exception_; -} - -DatabasePtr DatabaseMaterializeMySQL::getNestedDatabase() const +void DatabaseMaterializeMySQL::rethrowExceptionIfNeed() const { std::unique_lock lock(mutex); @@ -54,8 +48,12 @@ DatabasePtr DatabaseMaterializeMySQL::getNestedDatabase() const throw Exception(ex); } } +} - return nested_database; +void DatabaseMaterializeMySQL::setException(const std::exception_ptr & exception_) +{ + std::unique_lock lock(mutex); + exception = exception_; } ASTPtr DatabaseMaterializeMySQL::getCreateDatabaseQuery() const @@ -65,17 +63,21 @@ ASTPtr DatabaseMaterializeMySQL::getCreateDatabaseQuery() const create_query->set(create_query->storage, engine_define); return create_query; } -void DatabaseMaterializeMySQL::loadStoredObjects(Context & context, bool has_force_restore_data_flag) + +void DatabaseMaterializeMySQL::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) { try { - getNestedDatabase()->loadStoredObjects(context, has_force_restore_data_flag); + std::unique_lock lock(mutex); + nested_database->loadStoredObjects(context, has_force_restore_data_flag, force_attach); materialize_thread.startSynchronization(); } catch (...) { tryLogCurrentException(log, "Cannot load MySQL nested database stored objects."); - throw; + + if (!force_attach) + throw; } } @@ -92,42 +94,42 @@ void DatabaseMaterializeMySQL::shutdown() bool DatabaseMaterializeMySQL::empty() const { - return getNestedDatabase()->empty(); + return nested_database->empty(); } String DatabaseMaterializeMySQL::getDataPath() const { - return getNestedDatabase()->getDataPath(); + return nested_database->getDataPath(); } String DatabaseMaterializeMySQL::getMetadataPath() const { - return getNestedDatabase()->getMetadataPath(); + return nested_database->getMetadataPath(); } String DatabaseMaterializeMySQL::getTableDataPath(const String & table_name) const { - return getNestedDatabase()->getTableDataPath(table_name); + return nested_database->getTableDataPath(table_name); } String DatabaseMaterializeMySQL::getTableDataPath(const ASTCreateQuery & query) const { - return getNestedDatabase()->getTableDataPath(query); + return nested_database->getTableDataPath(query); } String DatabaseMaterializeMySQL::getObjectMetadataPath(const String & table_name) const { - return getNestedDatabase()->getObjectMetadataPath(table_name); + return nested_database->getObjectMetadataPath(table_name); } UUID DatabaseMaterializeMySQL::tryGetTableUUID(const String & table_name) const { - return getNestedDatabase()->tryGetTableUUID(table_name); + return nested_database->tryGetTableUUID(table_name); } time_t DatabaseMaterializeMySQL::getObjectMetadataModificationTime(const String & name) const { - return getNestedDatabase()->getObjectMetadataModificationTime(name); + return nested_database->getObjectMetadataModificationTime(name); } void DatabaseMaterializeMySQL::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) @@ -135,7 +137,7 @@ void DatabaseMaterializeMySQL::createTable(const Context & context, const String if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) throw Exception("MaterializeMySQL database not support create table.", ErrorCodes::NOT_IMPLEMENTED); - getNestedDatabase()->createTable(context, name, table, query); + nested_database->createTable(context, name, table, query); } void DatabaseMaterializeMySQL::dropTable(const Context & context, const String & name, bool no_delay) @@ -143,7 +145,7 @@ void DatabaseMaterializeMySQL::dropTable(const Context & context, const String & if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) throw Exception("MaterializeMySQL database not support drop table.", ErrorCodes::NOT_IMPLEMENTED); - getNestedDatabase()->dropTable(context, name, no_delay); + nested_database->dropTable(context, name, no_delay); } void DatabaseMaterializeMySQL::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) @@ -151,7 +153,7 @@ void DatabaseMaterializeMySQL::attachTable(const String & name, const StoragePtr if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) throw Exception("MaterializeMySQL database not support attach table.", ErrorCodes::NOT_IMPLEMENTED); - getNestedDatabase()->attachTable(name, table, relative_table_path); + nested_database->attachTable(name, table, relative_table_path); } StoragePtr DatabaseMaterializeMySQL::detachTable(const String & name) @@ -159,7 +161,7 @@ StoragePtr DatabaseMaterializeMySQL::detachTable(const String & name) if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) throw Exception("MaterializeMySQL database not support detach table.", ErrorCodes::NOT_IMPLEMENTED); - return getNestedDatabase()->detachTable(name); + return nested_database->detachTable(name); } void DatabaseMaterializeMySQL::renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) @@ -176,7 +178,7 @@ void DatabaseMaterializeMySQL::renameTable(const Context & context, const String if (to_database.getDatabaseName() != getDatabaseName()) throw Exception("Cannot rename with other database for MaterializeMySQL database.", ErrorCodes::NOT_IMPLEMENTED); - getNestedDatabase()->renameTable(context, name, *getNestedDatabase(), to_name, exchange, dictionary); + nested_database->renameTable(context, name, *nested_database, to_name, exchange, dictionary); } void DatabaseMaterializeMySQL::alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) @@ -184,7 +186,7 @@ void DatabaseMaterializeMySQL::alterTable(const Context & context, const Storage if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) throw Exception("MaterializeMySQL database not support alter table.", ErrorCodes::NOT_IMPLEMENTED); - getNestedDatabase()->alterTable(context, table_id, metadata); + nested_database->alterTable(context, table_id, metadata); } bool DatabaseMaterializeMySQL::shouldBeEmptyOnDetach() const @@ -194,14 +196,14 @@ bool DatabaseMaterializeMySQL::shouldBeEmptyOnDetach() const void DatabaseMaterializeMySQL::drop(const Context & context) { - DatabasePtr database = getNestedDatabase(); - - if (database->shouldBeEmptyOnDetach()) + if (nested_database->shouldBeEmptyOnDetach()) { - for (auto iterator = database->getTablesIterator(context, {}); iterator->isValid(); iterator->next()) + for (auto iterator = nested_database->getTablesIterator(context, {}); iterator->isValid(); iterator->next()) { - TableExclusiveLockHolder table_lock = iterator->table()->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - database->dropTable(context, iterator->name(), true); + TableExclusiveLockHolder table_lock = iterator->table()->lockExclusively( + context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + + nested_database->dropTable(context, iterator->name(), true); } /// Remove metadata info @@ -211,38 +213,38 @@ void DatabaseMaterializeMySQL::drop(const Context & context) metadata.remove(false); } - database->drop(context); + nested_database->drop(context); } bool DatabaseMaterializeMySQL::isTableExist(const String & name, const Context & context) const { - return getNestedDatabase()->isTableExist(name, context); + return nested_database->isTableExist(name, context); } StoragePtr DatabaseMaterializeMySQL::tryGetTable(const String & name, const Context & context) const { if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) { - StoragePtr nested_storage = getNestedDatabase()->tryGetTable(name, context); + StoragePtr nested_storage = nested_database->tryGetTable(name, context); if (!nested_storage) return {}; - return std::make_shared(std::move(nested_storage)); + return std::make_shared(std::move(nested_storage), this); } - return getNestedDatabase()->tryGetTable(name, context); + return nested_database->tryGetTable(name, context); } DatabaseTablesIteratorPtr DatabaseMaterializeMySQL::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) { if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) { - DatabaseTablesIteratorPtr iterator = getNestedDatabase()->getTablesIterator(context, filter_by_table_name); - return std::make_unique(std::move(iterator)); + DatabaseTablesIteratorPtr iterator = nested_database->getTablesIterator(context, filter_by_table_name); + return std::make_unique(std::move(iterator), this); } - return getNestedDatabase()->getTablesIterator(context, filter_by_table_name); + return nested_database->getTablesIterator(context, filter_by_table_name); } } diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h index c1fd0bb77a0..93548584296 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -25,6 +25,8 @@ public: const IAST * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, std::unique_ptr settings_); + void rethrowExceptionIfNeed() const; + void setException(const std::exception_ptr & exception); protected: const Context & global_context; @@ -38,14 +40,12 @@ protected: std::exception_ptr exception; - DatabasePtr getNestedDatabase() const; - public: - String getEngineName() const override { return "MySQL"; } + String getEngineName() const override { return "MaterializeMySQL"; } ASTPtr getCreateDatabaseQuery() const override; - void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override; + void loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) override; void shutdown() override; diff --git a/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h b/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h index 3bad8f9c8bb..86a5cbf8206 100644 --- a/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h +++ b/src/Databases/MySQL/DatabaseMaterializeTablesIterator.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -23,18 +24,21 @@ public: const StoragePtr & table() const override { - StoragePtr storage = std::make_shared(nested_iterator->table()); + StoragePtr storage = std::make_shared(nested_iterator->table(), database); return tables.emplace_back(storage); } UUID uuid() const override { return nested_iterator->uuid(); } - DatabaseMaterializeTablesIterator(DatabaseTablesIteratorPtr nested_iterator_) : nested_iterator(std::move(nested_iterator_)) - {} + DatabaseMaterializeTablesIterator(DatabaseTablesIteratorPtr nested_iterator_, DatabaseMaterializeMySQL * database_) + : nested_iterator(std::move(nested_iterator_)), database(database_) + { + } private: mutable std::vector tables; DatabaseTablesIteratorPtr nested_iterator; + DatabaseMaterializeMySQL * database; }; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index cd2e45323d1..7e5857c8440 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -209,7 +209,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) renamed = true; } - database->loadStoredObjects(context, has_force_restore_data_flag); + database->loadStoredObjects(context, has_force_restore_data_flag, create.attach && force_attach); } catch (...) { diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index bb7d70975e4..4a5d57c11d1 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -45,6 +45,11 @@ public: internal = internal_; } + void setForceAttach(bool force_attach_) + { + force_attach = force_attach_; + } + /// Obtain information about columns, their types, default values and column comments, /// for case when columns in CREATE query is specified explicitly. static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, const Context & context, bool sanity_check_compression_codecs); @@ -82,5 +87,6 @@ private: bool has_force_restore_data_flag = false; /// Is this an internal query - not from the user. bool internal = false; + bool force_attach = false; }; } diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index 62683dc546d..9fb71cf3a14 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -22,7 +22,6 @@ static inline ASTPtr tryRewrittenCreateQuery(const String & query, const Context ASTPtr ast = parseQuery(external_ddl_parser, query, 0, 0); Context context = context_; - context.unsafeSetCurrentDatabase("default"); return MySQLInterpreter::InterpreterCreateImpl::getRewrittenQuery( *ast->as()->external_ddl->as(), context, "test_database", "test_database"); diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 0bd97252090..3bcee18932e 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -38,6 +38,7 @@ static void executeCreateQuery( InterpreterCreateQuery interpreter(ast, context); interpreter.setInternal(true); + interpreter.setForceAttach(true); interpreter.setForceRestoreData(has_force_restore_data_flag); interpreter.execute(); } diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index 880571d4ddc..ae7745e945b 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -13,8 +13,8 @@ namespace DB { -StorageMaterializeMySQL::StorageMaterializeMySQL(const StoragePtr & nested_storage_) - : IStorage(nested_storage_->getStorageID()), nested_storage(nested_storage_) +StorageMaterializeMySQL::StorageMaterializeMySQL(const StoragePtr & nested_storage_, const DatabaseMaterializeMySQL * database_) + : IStorage(nested_storage_->getStorageID()), nested_storage(nested_storage_), database(database_) { ColumnsDescription columns_desc; const auto & nested_memory_metadata = nested_storage->getInMemoryMetadata(); @@ -38,6 +38,9 @@ Pipes StorageMaterializeMySQL::read( size_t max_block_size, unsigned int num_streams) { + /// If the background synchronization thread has exception. + database->rethrowExceptionIfNeed(); + NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); @@ -95,6 +98,9 @@ Pipes StorageMaterializeMySQL::read( NamesAndTypesList StorageMaterializeMySQL::getVirtuals() const { + /// If the background synchronization thread has exception. + database->rethrowExceptionIfNeed(); + NamesAndTypesList virtuals; Block nested_header = nested_storage->getInMemoryMetadata().getSampleBlockNonMaterialized(); ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); diff --git a/src/Storages/StorageMaterializeMySQL.h b/src/Storages/StorageMaterializeMySQL.h index e1b6aa12479..62909e01a36 100644 --- a/src/Storages/StorageMaterializeMySQL.h +++ b/src/Storages/StorageMaterializeMySQL.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -9,13 +10,13 @@ class StorageMaterializeMySQL final : public ext::shared_ptr_helper; public: - String getName() const override { return "MySQL"; } + String getName() const override { return "MaterializeMySQL"; } bool supportsFinal() const override { return nested_storage->supportsFinal(); } bool supportsSampling() const override { return nested_storage->supportsSampling(); } - StorageMaterializeMySQL(const StoragePtr & nested_storage_); + StorageMaterializeMySQL(const StoragePtr & nested_storage_, const DatabaseMaterializeMySQL * database_); Pipes read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -25,6 +26,7 @@ public: private: StoragePtr nested_storage; + const DatabaseMaterializeMySQL * database; }; } From bd21f8e6b9baae0399220fc98cad163482cbf483 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 7 Aug 2020 12:55:45 +0800 Subject: [PATCH 141/374] ISSUES-4006 fix conflicts commit when merge --- src/Core/ya.make | 2 ++ src/DataStreams/ya.make | 1 + src/Databases/MySQL/MaterializeMetadata.cpp | 7 ++++++- src/Databases/MySQL/MaterializeMetadata.h | 7 +++++++ src/Databases/MySQL/MaterializeMySQLSyncThread.h | 4 +++- src/Databases/ya.make | 6 +++++- src/Interpreters/InterpreterExternalDDLQuery.cpp | 2 +- .../MySQL/InterpretersMySQLDDLQuery.cpp | 6 +++--- .../MySQL/tests/gtest_create_rewritten.cpp | 3 +-- src/Interpreters/ya.make | 2 ++ src/Parsers/ya.make | 14 ++++++++++++++ src/Storages/ya.make | 1 + 12 files changed, 46 insertions(+), 9 deletions(-) diff --git a/src/Core/ya.make b/src/Core/ya.make index 197908300fe..3b45cf22f57 100644 --- a/src/Core/ya.make +++ b/src/Core/ya.make @@ -17,7 +17,9 @@ SRCS( ExternalTable.cpp Field.cpp iostream_debug_helpers.cpp + MySQLClient.cpp MySQLProtocol.cpp + MySQLReplication.cpp NamesAndTypes.cpp PostgreSQLProtocol.cpp Settings.cpp diff --git a/src/DataStreams/ya.make b/src/DataStreams/ya.make index a9bbd987ff0..dcada250d38 100644 --- a/src/DataStreams/ya.make +++ b/src/DataStreams/ya.make @@ -11,6 +11,7 @@ NO_COMPILER_WARNINGS() SRCS( AddingDefaultBlockOutputStream.cpp AddingDefaultsBlockInputStream.cpp + AddingVersionsBlockOutputStream.cpp AsynchronousBlockInputStream.cpp BlockIO.cpp BlockStreamProfileInfo.cpp diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 0b470dda903..80484c130d3 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -1,7 +1,10 @@ +#include + +#if USE_MYSQL + #include #include #include -#include #include #include #include @@ -206,3 +209,5 @@ MaterializeMetadata::MaterializeMetadata( } } + +#endif diff --git a/src/Databases/MySQL/MaterializeMetadata.h b/src/Databases/MySQL/MaterializeMetadata.h index c377a99a27c..045dab5b84c 100644 --- a/src/Databases/MySQL/MaterializeMetadata.h +++ b/src/Databases/MySQL/MaterializeMetadata.h @@ -1,5 +1,11 @@ #pragma once +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#if USE_MYSQL + #include #include #include @@ -42,3 +48,4 @@ struct MaterializeMetadata } +#endif diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index f244c24ab9a..9c454de2a3f 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -1,6 +1,8 @@ #pragma once -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif #if USE_MYSQL diff --git a/src/Databases/ya.make b/src/Databases/ya.make index 1fc21caf340..6af786578f8 100644 --- a/src/Databases/ya.make +++ b/src/Databases/ya.make @@ -11,11 +11,15 @@ SRCS( DatabaseFactory.cpp DatabaseLazy.cpp DatabaseMemory.cpp - DatabaseMySQL.cpp DatabaseOnDisk.cpp DatabaseOrdinary.cpp DatabasesCommon.cpp DatabaseWithDictionaries.cpp + MySQL/DatabaseConnectionMySQL.cpp + MySQL/DatabaseMaterializeMySQL.cpp + MySQL/MaterializeMetadata.cpp + MySQL/MaterializeMySQLSettings.cpp + MySQL/MaterializeMySQLSyncThread.cpp ) diff --git a/src/Interpreters/InterpreterExternalDDLQuery.cpp b/src/Interpreters/InterpreterExternalDDLQuery.cpp index 89b8e598ac9..4a93c0fa753 100644 --- a/src/Interpreters/InterpreterExternalDDLQuery.cpp +++ b/src/Interpreters/InterpreterExternalDDLQuery.cpp @@ -3,6 +3,7 @@ #endif #include +#include #include #include @@ -11,7 +12,6 @@ #include #ifdef USE_MYSQL -# include # include # include # include diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index ca39a8ae4fc..95bd8c44d30 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -165,7 +165,7 @@ static inline std::tuplechanges.count("primary_key")) primary_keys->arguments->children.emplace_back(std::make_shared(declare_column->name)); - if (options->changes.contains("auto_increment")) + if (options->changes.count("auto_increment")) increment_columns.emplace(declare_column->name); } } @@ -272,10 +272,10 @@ static ASTPtr getOrderByPolicy( for (const auto & [name, type] : names_and_types) { - if (order_by_columns_set.contains(name)) + if (order_by_columns_set.count(name)) continue; - if (increment_columns.contains(name)) + if (increment_columns.count(name)) { increment_keys.emplace_back(name); order_by_columns_set.emplace(name); diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index 9fb71cf3a14..3782d20bd3d 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -16,12 +16,11 @@ using namespace DB; -static inline ASTPtr tryRewrittenCreateQuery(const String & query, const Context & context_) +static inline ASTPtr tryRewrittenCreateQuery(const String & query, const Context & context) { ParserExternalDDLQuery external_ddl_parser; ASTPtr ast = parseQuery(external_ddl_parser, query, 0, 0); - Context context = context_; return MySQLInterpreter::InterpreterCreateImpl::getRewrittenQuery( *ast->as()->external_ddl->as(), context, "test_database", "test_database"); diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index b543319dfa3..23cde61a744 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -78,6 +78,7 @@ SRCS( InterpreterDropQuery.cpp InterpreterExistsQuery.cpp InterpreterExplainQuery.cpp + InterpreterExternalDDLQuery.cpp InterpreterFactory.cpp InterpreterGrantQuery.cpp InterpreterInsertQuery.cpp @@ -110,6 +111,7 @@ SRCS( MergeJoin.cpp MetricLog.cpp MutationsInterpreter.cpp + MySQL/InterpretersMySQLDDLQuery.cpp NullableUtils.cpp OptimizeIfChains.cpp OptimizeIfWithConstantConditionVisitor.cpp diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make index ada5975bcf0..a2cc84223a7 100644 --- a/src/Parsers/ya.make +++ b/src/Parsers/ya.make @@ -66,6 +66,19 @@ SRCS( IParserBase.cpp Lexer.cpp makeASTForLogicalFunction.cpp + MySQL/ASTAlterCommand.cpp + MySQL/ASTAlterQuery.cpp + MySQL/ASTCreateDefines.cpp + MySQL/ASTCreateQuery.cpp + MySQL/ASTDeclareColumn.cpp + MySQL/ASTDeclareConstraint.cpp + MySQL/ASTDeclareIndex.cpp + MySQL/ASTDeclareOption.cpp + MySQL/ASTDeclarePartition.cpp + MySQL/ASTDeclarePartitionOptions.cpp + MySQL/ASTDeclareReference.cpp + MySQL/ASTDeclareSubPartition.cpp + MySQL/ASTDeclareTableOptions.cpp parseDatabaseAndTableName.cpp parseIdentifierOrStringLiteral.cpp parseIntervalKind.cpp @@ -86,6 +99,7 @@ SRCS( ParserDropAccessEntityQuery.cpp ParserDropQuery.cpp ParserExplainQuery.cpp + ParserExternalDDLQuery.cpp ParserGrantQuery.cpp ParserInsertQuery.cpp ParserKillQueryQuery.cpp diff --git a/src/Storages/ya.make b/src/Storages/ya.make index a6c7fc5e4fe..8b60bd34a36 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -115,6 +115,7 @@ SRCS( StorageLog.cpp StorageLogSettings.cpp StorageMaterializedView.cpp + StorageMaterializeMySQL.cpp StorageMemory.cpp StorageMerge.cpp StorageMergeTree.cpp From d6bba8cbbb85529a8efba946013c8dfa095b6d9f Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 7 Aug 2020 14:04:50 +0800 Subject: [PATCH 142/374] ISSUES-4006 add settings for allows_query_when_mysql_lost & move dump event to logger --- src/Core/MySQLReplication.cpp | 98 +++++++++---------- src/Core/MySQLReplication.h | 18 ++-- src/Core/tests/mysql_protocol.cpp | 8 +- .../MySQL/DatabaseMaterializeMySQL.cpp | 2 +- .../MySQL/MaterializeMySQLSettings.h | 3 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 14 ++- 6 files changed, 76 insertions(+), 67 deletions(-) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index 934a262b36c..db0c70d07a1 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -27,15 +27,15 @@ namespace MySQLReplication payload.readStrict(reinterpret_cast(&flags), 2); } - void EventHeader::dump() const + void EventHeader::dump(std::ostream & out) const { - std::cerr << "\n=== " << to_string(this->type) << " ===" << std::endl; - std::cerr << "Timestamp: " << this->timestamp << std::endl; - std::cerr << "Event Type: " << this->type << std::endl; - std::cerr << "Server ID: " << this->server_id << std::endl; - std::cerr << "Event Size: " << this->event_size << std::endl; - std::cerr << "Log Pos: " << this->log_pos << std::endl; - std::cerr << "Flags: " << this->flags << std::endl; + out << "\n=== " << to_string(this->type) << " ===" << std::endl; + out << "Timestamp: " << this->timestamp << std::endl; + out << "Event Type: " << this->type << std::endl; + out << "Server ID: " << this->server_id << std::endl; + out << "Event Size: " << this->event_size << std::endl; + out << "Log Pos: " << this->log_pos << std::endl; + out << "Flags: " << this->flags << std::endl; } /// https://dev.mysql.com/doc/internals/en/format-description-event.html @@ -53,13 +53,13 @@ namespace MySQLReplication payload.readStrict(reinterpret_cast(event_type_header_length.data()), len); } - void FormatDescriptionEvent::dump() const + void FormatDescriptionEvent::dump(std::ostream & out) const { - header.dump(); - std::cerr << "Binlog Version: " << this->binlog_version << std::endl; - std::cerr << "Server Version: " << this->server_version << std::endl; - std::cerr << "Create Timestamp: " << this->create_timestamp << std::endl; - std::cerr << "Event Header Len: " << this->event_header_length << std::endl; + header.dump(out); + out << "Binlog Version: " << this->binlog_version << std::endl; + out << "Server Version: " << this->server_version << std::endl; + out << "Create Timestamp: " << this->create_timestamp << std::endl; + out << "Event Header Len: " << this->event_header_length << std::endl; } /// https://dev.mysql.com/doc/internals/en/rotate-event.html @@ -71,11 +71,11 @@ namespace MySQLReplication payload.readStrict(reinterpret_cast(next_binlog.data()), len); } - void RotateEvent::dump() const + void RotateEvent::dump(std::ostream & out) const { - header.dump(); - std::cerr << "Position: " << this->position << std::endl; - std::cerr << "Next Binlog: " << this->next_binlog << std::endl; + header.dump(out); + out << "Position: " << this->position << std::endl; + out << "Next Binlog: " << this->next_binlog << std::endl; } /// https://dev.mysql.com/doc/internals/en/query-event.html @@ -114,25 +114,25 @@ namespace MySQLReplication } } - void QueryEvent::dump() const + void QueryEvent::dump(std::ostream & out) const { - header.dump(); - std::cerr << "Thread ID: " << this->thread_id << std::endl; - std::cerr << "Execution Time: " << this->exec_time << std::endl; - std::cerr << "Schema Len: " << this->schema_len << std::endl; - std::cerr << "Error Code: " << this->error_code << std::endl; - std::cerr << "Status Len: " << this->status_len << std::endl; - std::cerr << "Schema: " << this->schema << std::endl; - std::cerr << "Query: " << this->query << std::endl; + header.dump(out); + out << "Thread ID: " << this->thread_id << std::endl; + out << "Execution Time: " << this->exec_time << std::endl; + out << "Schema Len: " << this->schema_len << std::endl; + out << "Error Code: " << this->error_code << std::endl; + out << "Status Len: " << this->status_len << std::endl; + out << "Schema: " << this->schema << std::endl; + out << "Query: " << this->query << std::endl; } void XIDEvent::parseImpl(ReadBuffer & payload) { payload.readStrict(reinterpret_cast(&xid), 8); } - void XIDEvent::dump() const + void XIDEvent::dump(std::ostream & out) const { - header.dump(); - std::cerr << "XID: " << this->xid << std::endl; + header.dump(out); + out << "XID: " << this->xid << std::endl; } void TableMapEvent::parseImpl(ReadBuffer & payload) @@ -234,21 +234,21 @@ namespace MySQLReplication } } - void TableMapEvent::dump() const + void TableMapEvent::dump(std::ostream & out) const { - header.dump(); - std::cerr << "Table ID: " << this->table_id << std::endl; - std::cerr << "Flags: " << this->flags << std::endl; - std::cerr << "Schema Len: " << this->schema_len << std::endl; - std::cerr << "Schema: " << this->schema << std::endl; - std::cerr << "Table Len: " << this->table_len << std::endl; - std::cerr << "Table: " << this->table << std::endl; - std::cerr << "Column Count: " << this->column_count << std::endl; + header.dump(out); + out << "Table ID: " << this->table_id << std::endl; + out << "Flags: " << this->flags << std::endl; + out << "Schema Len: " << this->schema_len << std::endl; + out << "Schema: " << this->schema << std::endl; + out << "Table Len: " << this->table_len << std::endl; + out << "Table: " << this->table << std::endl; + out << "Column Count: " << this->column_count << std::endl; for (auto i = 0U; i < column_count; i++) { - std::cerr << "Column Type [" << i << "]: " << column_type[i] << ", Meta: " << column_meta[i] << std::endl; + out << "Column Type [" << i << "]: " << column_type[i] << ", Meta: " << column_meta[i] << std::endl; } - std::cerr << "Null Bitmap: " << this->null_bitmap << std::endl; + out << "Null Bitmap: " << this->null_bitmap << std::endl; } void RowsEvent::parseImpl(ReadBuffer & payload) @@ -688,25 +688,25 @@ namespace MySQLReplication rows.push_back(row); } - void RowsEvent::dump() const + void RowsEvent::dump(std::ostream & out) const { FieldVisitorToString to_string; - header.dump(); - std::cerr << "Schema: " << this->schema << std::endl; - std::cerr << "Table: " << this->table << std::endl; + header.dump(out); + out << "Schema: " << this->schema << std::endl; + out << "Table: " << this->table << std::endl; for (auto i = 0U; i < rows.size(); i++) { - std::cerr << "Row[" << i << "]: " << applyVisitor(to_string, rows[i]) << std::endl; + out << "Row[" << i << "]: " << applyVisitor(to_string, rows[i]) << std::endl; } } void DryRunEvent::parseImpl(ReadBuffer & payload) { payload.ignore(header.event_size - EVENT_HEADER_LENGTH); } - void DryRunEvent::dump() const + void DryRunEvent::dump(std::ostream & out) const { - header.dump(); - std::cerr << "[DryRun Event]" << std::endl; + header.dump(out); + out << "[DryRun Event]" << std::endl; } void MySQLFlavor::readPayloadImpl(ReadBuffer & payload) diff --git a/src/Core/MySQLReplication.h b/src/Core/MySQLReplication.h index f4ac02c2ffd..29af875a430 100644 --- a/src/Core/MySQLReplication.h +++ b/src/Core/MySQLReplication.h @@ -289,7 +289,7 @@ namespace MySQLReplication UInt32 log_pos; UInt16 flags; - void dump() const; + void dump(std::ostream & out) const; void parse(ReadBuffer & payload); }; @@ -299,7 +299,7 @@ namespace MySQLReplication EventHeader header; virtual ~EventBase() = default; - virtual void dump() const = 0; + virtual void dump(std::ostream & out) const = 0; virtual void parseHeader(ReadBuffer & payload) { header.parse(payload); } virtual void parseEvent(ReadBuffer & payload) { parseImpl(payload); } virtual MySQLEventType type() const { return MYSQL_UNHANDLED_EVENT; } @@ -317,7 +317,7 @@ namespace MySQLReplication UInt8 event_header_length; String event_type_header_length; - void dump() const override; + void dump(std::ostream & out) const override; void parseImpl(ReadBuffer & payload) override; private: @@ -330,7 +330,7 @@ namespace MySQLReplication UInt64 position; String next_binlog; - void dump() const override; + void dump(std::ostream & out) const override; protected: void parseImpl(ReadBuffer & payload) override; @@ -356,7 +356,7 @@ namespace MySQLReplication String query; QueryType typ = DDL; - void dump() const override; + void dump(std::ostream & out) const override; MySQLEventType type() const override { return MYSQL_QUERY_EVENT; } protected: @@ -368,7 +368,7 @@ namespace MySQLReplication protected: UInt64 xid; - void dump() const override; + void dump(std::ostream & out) const override; void parseImpl(ReadBuffer & payload) override; }; @@ -386,7 +386,7 @@ namespace MySQLReplication std::vector column_meta; Bitmap null_bitmap; - void dump() const override; + void dump(std::ostream & out) const override; protected: void parseImpl(ReadBuffer & payload) override; @@ -408,7 +408,7 @@ namespace MySQLReplication table = table_map->table; } - void dump() const override; + void dump(std::ostream & out) const override; protected: UInt64 table_id; @@ -447,7 +447,7 @@ namespace MySQLReplication class DryRunEvent : public EventBase { - void dump() const override; + void dump(std::ostream & out) const override; protected: void parseImpl(ReadBuffer & payload) override; diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index c86cf0be25a..e5e875b1975 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -205,7 +205,7 @@ int main(int argc, char ** argv) { case MYSQL_QUERY_EVENT: { auto binlog_event = std::static_pointer_cast(event); - binlog_event->dump(); + binlog_event->dump(std::cerr); Position pos = slave.getPosition(); std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; @@ -213,7 +213,7 @@ int main(int argc, char ** argv) } case MYSQL_WRITE_ROWS_EVENT: { auto binlog_event = std::static_pointer_cast(event); - binlog_event->dump(); + binlog_event->dump(std::cerr); Position pos = slave.getPosition(); std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; @@ -221,7 +221,7 @@ int main(int argc, char ** argv) } case MYSQL_UPDATE_ROWS_EVENT: { auto binlog_event = std::static_pointer_cast(event); - binlog_event->dump(); + binlog_event->dump(std::cerr); Position pos = slave.getPosition(); std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; @@ -229,7 +229,7 @@ int main(int argc, char ** argv) } case MYSQL_DELETE_ROWS_EVENT: { auto binlog_event = std::static_pointer_cast(event); - binlog_event->dump(); + binlog_event->dump(std::cerr); Position pos = slave.getPosition(); std::cerr << "Binlog Name: " << pos.binlog_name << ", Pos: " << pos.binlog_pos << std::endl; diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 1d3419a218c..ce21d312b1f 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -37,7 +37,7 @@ void DatabaseMaterializeMySQL::rethrowExceptionIfNeed() const { std::unique_lock lock(mutex); - if (exception) + if (!settings->allows_query_when_mysql_lost && exception) { try { diff --git a/src/Databases/MySQL/MaterializeMySQLSettings.h b/src/Databases/MySQL/MaterializeMySQLSettings.h index 539471becfb..07de219c72f 100644 --- a/src/Databases/MySQL/MaterializeMySQLSettings.h +++ b/src/Databases/MySQL/MaterializeMySQLSettings.h @@ -14,7 +14,8 @@ class ASTStorage; M(UInt64, max_rows_in_buffers, DEFAULT_BLOCK_SIZE, "Max rows that data is allowed to cache in memory(for database and the cache data unable to query). when rows is exceeded, the data will be materialized", 0) \ M(UInt64, max_bytes_in_buffers, DBMS_DEFAULT_BUFFER_SIZE, "Max bytes that data is allowed to cache in memory(for database and the cache data unable to query). when rows is exceeded, the data will be materialized", 0) \ M(UInt64, max_flush_data_time, 1000, "Max milliseconds that data is allowed to cache in memory(for database and the cache data unable to query). when this time is exceeded, the data will be materialized", 0) \ - M(UInt64, max_wait_time_when_mysql_unavailable, 1000, "", 0) \ + M(UInt64, max_wait_time_when_mysql_unavailable, 1000, "Dump full data retry interval when MySQL is not available(milliseconds).", 0) \ + M(Bool, allows_query_when_mysql_lost, false, "Allow query materialized table when mysql is lost.", 0) \ DECLARE_SETTINGS_TRAITS(MaterializeMySQLSettingsTraits, LIST_OF_MATERIALIZE_MODE_SETTINGS) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 43d1295e5c4..ea95f204a61 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -171,10 +171,7 @@ void MaterializeMySQLSyncThread::synchronization(const String & mysql_version) std::unique_lock lock(sync_mutex); if (binlog_event) - { - binlog_event->dump(); onEvent(buffers, binlog_event, *metadata); - } if (watch.elapsedMilliseconds() > max_flush_time || buffers.checkThresholds( settings->max_rows_in_buffer, settings->max_bytes_in_buffer, @@ -593,6 +590,17 @@ void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr throw; } } + else if (receive_event->header.type != HEARTBEAT_EVENT) + { + const auto & dump_event_message = [&]() + { + std::stringstream ss; + receive_event->dump(ss); + return ss.str(); + }; + + LOG_DEBUG(log, "Skip MySQL event: \n {}", dump_event_message()); + } } bool MaterializeMySQLSyncThread::isMySQLSyncThread() From cb879f4cf5665c28cacfb2c76af8340eaac5cc33 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 7 Aug 2020 11:28:12 +0300 Subject: [PATCH 143/374] Push ci --- src/Processors/QueryPipeline.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 751f93ad694..41ab4ad392d 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -415,6 +415,7 @@ void QueryPipeline::setCollectedProcessors(Processors * processors) pipe.collected_processors = processors; } + QueryPipelineProcessorsCollector::QueryPipelineProcessorsCollector(QueryPipeline & pipeline_, IQueryPlanStep * step_) : pipeline(pipeline_), step(step_) { From 6fca8ad7df966ed769920b71e08793c46c78672a Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 7 Aug 2020 16:49:27 +0800 Subject: [PATCH 144/374] ISSUES-4006 fix segfault on ClickHouse shutdown when MySQL is not available --- src/Databases/MySQL/MaterializeMySQLSyncThread.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index ea95f204a61..747adc244f4 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -196,7 +196,7 @@ void MaterializeMySQLSyncThread::synchronization(const String & mysql_version) void MaterializeMySQLSyncThread::stopSynchronization() { - if (!sync_quit) + if (!sync_quit && background_thread_pool) { { sync_quit = true; From 5de2491eda886fd254d3835b1960446482df9655 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 7 Aug 2020 11:55:51 +0300 Subject: [PATCH 145/374] Enable mixed granularity parts by default --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 9675c6e303c..9de3350e8d9 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -85,7 +85,7 @@ struct Settings; M(Int64, merge_with_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with TTL can be repeated.", 0) \ M(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \ M(Bool, write_final_mark, 1, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)", 0) \ - M(Bool, enable_mixed_granularity_parts, 0, "Enable parts with adaptive and non adaptive granularity", 0) \ + M(Bool, enable_mixed_granularity_parts, 1, "Enable parts with adaptive and non adaptive granularity", 0) \ M(MaxThreads, max_part_loading_threads, 0, "The number of threads to load data parts at startup.", 0) \ M(MaxThreads, max_part_removal_threads, 0, "The number of threads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).", 0) \ M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ From a6ee09f696c1744febb8ef48f1aee7885ba4f2df Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 7 Aug 2020 14:40:19 +0300 Subject: [PATCH 146/374] Move parts to DiskS3 in parallel refactoring. --- src/Disks/DiskAsyncSupport.h | 15 --- src/Disks/Executor.h | 17 +++ src/Disks/IDisk.cpp | 112 +++++++++---------- src/Disks/IDisk.h | 5 + src/Disks/S3/DiskS3.cpp | 58 ++++++---- src/Disks/S3/DiskS3.h | 10 +- src/Disks/S3/registerDiskS3.cpp | 1 - src/Interpreters/Context.cpp | 2 +- src/Interpreters/Context.h | 2 +- tests/integration/test_merge_tree_s3/test.py | 4 + 10 files changed, 119 insertions(+), 107 deletions(-) delete mode 100644 src/Disks/DiskAsyncSupport.h create mode 100644 src/Disks/Executor.h diff --git a/src/Disks/DiskAsyncSupport.h b/src/Disks/DiskAsyncSupport.h deleted file mode 100644 index 14be095843e..00000000000 --- a/src/Disks/DiskAsyncSupport.h +++ /dev/null @@ -1,15 +0,0 @@ -#pragma once - -#include "future" - -namespace DB -{ - -class DiskAsyncSupport -{ -public: - virtual ~DiskAsyncSupport() = default; - virtual std::future runAsync(std::function task) = 0; -}; - -} diff --git a/src/Disks/Executor.h b/src/Disks/Executor.h new file mode 100644 index 00000000000..c47201c402d --- /dev/null +++ b/src/Disks/Executor.h @@ -0,0 +1,17 @@ +#pragma once + +#include "future" +#include "functional" + +namespace DB +{ + +/// Interface to run task asynchronously with possibility to wait for execution. +class Executor +{ +public: + virtual ~Executor() = default; + virtual std::future execute(std::function task) = 0; +}; + +} diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index b4ad9c4b643..9a225e8da9e 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -1,11 +1,12 @@ #include "IDisk.h" -#include +#include "Disks/Executor.h" #include #include +#include #include #include +#include #include "future" -#include "Disks/DiskAsyncSupport.h" namespace DB { @@ -20,28 +21,6 @@ bool IDisk::isDirectoryEmpty(const String & path) return !iterateDirectory(path)->isValid(); } -/// Executes task synchronously in case when destination disk doesn't support async operations. -class NoAsync : public DiskAsyncSupport -{ -public: - NoAsync() = default; - ~NoAsync() override = default; - std::future runAsync(std::function task) override - { - std::promise promise; - try - { - task(); - promise.set_value(); - } - catch (...) - { - promise.set_exception(std::current_exception()); - } - return promise.get_future(); - } -}; - void copyFile(IDisk & from_disk, const String & from_path, IDisk & to_disk, const String & to_path) { LOG_DEBUG(&Poco::Logger::get("IDisk"), "Copying from {} {} to {} {}.", from_disk.getName(), from_path, to_disk.getName(), to_path); @@ -51,16 +30,21 @@ void copyFile(IDisk & from_disk, const String & from_path, IDisk & to_disk, cons copyData(*in, *out); } -std::future asyncCopy(IDisk & from_disk, const String & from_path, IDisk & to_disk, const String & to_path, DiskAsyncSupport & async) + +using ResultsCollector = std::vector>; + +void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_path, Executor & exec, ResultsCollector & results) { if (from_disk.isFile(from_path)) { - return async.runAsync( - [&from_disk, &from_path, &to_disk, &to_path]() + auto result = exec.execute( + [&from_disk, from_path, &to_disk, to_path]() { + setThreadName("DiskCopier"); DB::copyFile(from_disk, from_path, to_disk, to_path + fileName(from_path)); - } - ); + }); + + results.push_back(std::move(result)); } else { @@ -69,44 +53,52 @@ std::future asyncCopy(IDisk & from_disk, const String & from_path, IDisk & const String dest = to_path + dir_name + "/"; to_disk.createDirectories(dest); - std::vector> futures; - std::promise promise; - for (auto it = from_disk.iterateDirectory(from_path); it->isValid(); it->next()) - futures.push_back(asyncCopy(from_disk, it->path(), to_disk, dest, async)); - - for (auto & future : futures) - { - future.wait(); - try - { - future.get(); - } - catch (...) - { - promise.set_exception(std::current_exception()); - } - } - - return promise.get_future(); + asyncCopy(from_disk, it->path(), to_disk, dest, exec, results); } } void IDisk::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) { - std::future future; - if (auto async = std::dynamic_pointer_cast(to_disk)) - { - future = asyncCopy(*this, from_path, *to_disk, to_path, *async); - } - else - { - auto no_async = std::make_unique(); - future = asyncCopy(*this, from_path, *to_disk, to_path, *no_async); - } + auto exec = to_disk->getExecutor(); + ResultsCollector results; - future.wait(); - future.get(); + asyncCopy(*this, from_path, *to_disk, to_path, *exec, results); + + for (auto & result : results) + result.wait(); + for (auto & result : results) + result.get(); +} + +/// Executes task synchronously in case when disk doesn't support async operations. +class SyncExecutor : public Executor +{ +public: + SyncExecutor() = default; + std::future execute(std::function task) override + { + auto promise = std::make_shared>(); + try + { + task(); + promise->set_value(); + } + catch (...) + { + try + { + promise->set_exception(std::current_exception()); + } + catch (...) { } + } + return promise->get_future(); + } +}; + +std::unique_ptr IDisk::getExecutor() +{ + return std::make_unique(); } void IDisk::truncateFile(const String &, size_t) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index ea331c5f64b..f59c8551633 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -29,6 +29,7 @@ using Reservations = std::vector; class ReadBufferFromFileBase; class WriteBufferFromFileBase; +class Executor; /** * Mode of opening a file for write. @@ -178,6 +179,10 @@ public: /// Return disk type - "local", "s3", etc. virtual const String getType() const = 0; + +private: + /// Returns executor to perform asynchronous operations. + virtual std::unique_ptr getExecutor(); }; using DiskPtr = std::shared_ptr; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index bfa1d881b8e..f03e937a290 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -416,12 +416,44 @@ private: CurrentMetrics::Increment metric_increment; }; +/// Runs tasks asynchronously using global thread pool. +class AsyncExecutor : public Executor +{ +public: + explicit AsyncExecutor() = default; + + std::future execute(std::function task) + { + auto promise = std::make_shared>(); + + GlobalThreadPool::instance().scheduleOrThrowOnError( + [promise, task]() + { + try + { + task(); + promise->set_value(); + } + catch (...) + { + tryLogCurrentException(&Poco::Logger::get("DiskS3"), "Failed to run async task"); + + try + { + promise->set_exception(std::current_exception()); + } catch (...) { } + } + }); + + return promise->get_future(); + } +}; + DiskS3::DiskS3( String name_, std::shared_ptr client_, std::shared_ptr proxy_configuration_, - BackgroundProcessingPool & pool_, String bucket_, String s3_root_path_, String metadata_path_, @@ -431,7 +463,6 @@ DiskS3::DiskS3( : name(std::move(name_)) , client(std::move(client_)) , proxy_configuration(std::move(proxy_configuration_)) - , pool(pool_) , bucket(std::move(bucket_)) , s3_root_path(std::move(s3_root_path_)) , metadata_path(std::move(metadata_path_)) @@ -691,28 +722,9 @@ void DiskS3::setReadOnly(const String & path) Poco::File(metadata_path + path).setReadOnly(true); } -std::future DiskS3::runAsync(std::function task) +std::unique_ptr DiskS3::getExecutor() { - std::promise promise; - - pool.addTask( - [&promise, &task]() - { - try - { - LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Executing async task..."); - - task(); - promise.set_value(); - } - catch (...) - { - promise.set_exception(std::current_exception()); - } - return BackgroundProcessingPoolTaskResult::SUCCESS; - }); - - return promise.get_future(); + return std::make_unique(); } DiskS3Reservation::~DiskS3Reservation() diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 053cc5e0e38..10ac5be0f65 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -1,7 +1,7 @@ #pragma once #include "Disks/DiskFactory.h" -#include "Disks/DiskAsyncSupport.h" +#include "Disks/Executor.h" #include "ProxyConfiguration.h" #include @@ -17,7 +17,7 @@ class BackgroundProcessingPool; * Files are represented by file in local filesystem (clickhouse_root/disks/disk_name/path/to/file) * that contains S3 object key with actual data. */ -class DiskS3 : public IDisk, public DiskAsyncSupport +class DiskS3 : public IDisk { public: friend class DiskS3Reservation; @@ -26,7 +26,6 @@ public: String name_, std::shared_ptr client_, std::shared_ptr proxy_configuration_, - BackgroundProcessingPool & pool, String bucket_, String s3_root_path_, String metadata_path_, @@ -104,16 +103,15 @@ public: const String getType() const override { return "s3"; } - std::future runAsync(std::function task) override; - private: bool tryReserve(UInt64 bytes); + std::unique_ptr getExecutor() override; + private: const String name; std::shared_ptr client; std::shared_ptr proxy_configuration; - BackgroundProcessingPool & pool; const String bucket; const String s3_root_path; const String metadata_path; diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 272a6e24b97..34c28a62477 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -136,7 +136,6 @@ void registerDiskS3(DiskFactory & factory) name, client, proxy_config, - context.getBackgroundMovePool(), uri.bucket, uri.key, metadata_path, diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4d7f8b93e1f..01097a09cbb 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1362,7 +1362,7 @@ BackgroundProcessingPool & Context::getBackgroundPool() return *shared->background_pool; } -BackgroundProcessingPool & Context::getBackgroundMovePool() const +BackgroundProcessingPool & Context::getBackgroundMovePool() { auto lock = getLock(); if (!shared->background_move_pool) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 830c5fc6bdb..dfe2d00873f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -499,7 +499,7 @@ public: BackgroundSchedulePool & getBufferFlushSchedulePool(); BackgroundProcessingPool & getBackgroundPool(); - BackgroundProcessingPool & getBackgroundMovePool() const; + BackgroundProcessingPool & getBackgroundMovePool(); BackgroundSchedulePool & getSchedulePool(); BackgroundSchedulePool & getDistributedSchedulePool(); diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 01923293b21..8a846ceb8a0 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -225,6 +225,10 @@ def test_move_partition_to_another_disk(cluster): assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)" assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE + node.query("ALTER TABLE s3_test MOVE PARTITION '2020-01-04' TO DISK 's3'") + assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)" + assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE*2 + def test_table_manipulations(cluster): create_table(cluster, "s3_test") From 53ecd95302c173b06796e126cc0851ad1a8e200f Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 7 Aug 2020 14:44:35 +0300 Subject: [PATCH 147/374] Remove unused dependencies. --- src/Disks/S3/DiskS3.cpp | 1 - src/Disks/S3/DiskS3.h | 1 - 2 files changed, 2 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index f03e937a290..66db32a4837 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -16,7 +16,6 @@ #include #include #include -#include #include #include diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 10ac5be0f65..ac576666f2e 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -10,7 +10,6 @@ namespace DB { -class BackgroundProcessingPool; /** * Storage for persisting data in S3 and metadata on the local disk. From c9f81125bc757a76a0f6bbd355c46660eeffe765 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 7 Aug 2020 16:02:28 +0300 Subject: [PATCH 148/374] Review fixes. --- src/Storages/StorageMemory.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index e098a5ace4e..aaa8c56f9d9 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -23,8 +23,8 @@ class MemorySource : public SourceWithProgress { public: /// We use range [first, last] which includes right border. - /// It is needed because new elements may be added to list in other thread. - /// Appending of new element changes end() of std::list, which may cause data race otherwise. + /// Blocks are stored in std::list which may be appended in another thread. + /// We don't use synchronisation here, because elements in range [first, last] won't be modified. MemorySource( Names column_names_, BlocksList::iterator first_, @@ -33,7 +33,7 @@ public: const StorageMetadataPtr & metadata_snapshot) : SourceWithProgress(metadata_snapshot->getSampleBlockForColumns(column_names_, storage.getVirtuals(), storage.getStorageID())) , column_names(std::move(column_names_)) - , first(first_) + , current(first_) , last(last_) /// [first, last] { } @@ -49,7 +49,7 @@ protected: } else { - Block src = *first; + const Block & src = *current; Columns columns; columns.reserve(column_names.size()); @@ -57,16 +57,16 @@ protected: for (const auto & name : column_names) columns.emplace_back(src.getByName(name).column); - if (first == last) + if (current == last) is_finished = true; else - ++first; + ++current; return Chunk(std::move(columns), src.rows()); } } private: Names column_names; - BlocksList::iterator first; + BlocksList::iterator current; BlocksList::iterator last; bool is_finished = false; }; From 7315eac887c622f6df2e2f90c3e2f1bd076d78cd Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 7 Aug 2020 16:19:51 +0300 Subject: [PATCH 149/374] Update CHANGELOG.md --- CHANGELOG.md | 32 +------------------------------- 1 file changed, 1 insertion(+), 31 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e0ec3c58b5a..a8783f2042f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,7 +1,7 @@ ## ClickHouse release 20.6 -### ClickHouse release v20.6.2.15-prestable FIXME as compared to v20.5.4.40-stable +## ClickHouse release v20.6.2.15-prestable FIXME as compared to v20.5.4.40-stable #### New Feature @@ -127,41 +127,11 @@ * Install `ca-certificates` before the first `apt-get update` in Dockerfile. [#12095](https://github.com/ClickHouse/ClickHouse/pull/12095) ([Ivan Blinkov](https://github.com/blinkov)). * Add new type of tests based on Testflows framework. [#12090](https://github.com/ClickHouse/ClickHouse/pull/12090) ([vzakaznikov](https://github.com/vzakaznikov)). -#### NO CL ENTRY - -* NO CL ENTRY: 'Backport [#12700](https://github.com/ClickHouse/ClickHouse/issues/12700) to 20.6: Fix performance with large tuples'. [#13187](https://github.com/ClickHouse/ClickHouse/pull/13187) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#13075](https://github.com/ClickHouse/ClickHouse/issues/13075) to 20.6: Fix 11327'. [#13184](https://github.com/ClickHouse/ClickHouse/pull/13184) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#13081](https://github.com/ClickHouse/ClickHouse/issues/13081) to 20.6: Fix wrong index analysis with functions'. [#13146](https://github.com/ClickHouse/ClickHouse/pull/13146) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#13009](https://github.com/ClickHouse/ClickHouse/issues/13009) to 20.6: Fix 12623'. [#13051](https://github.com/ClickHouse/ClickHouse/pull/13051) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12989](https://github.com/ClickHouse/ClickHouse/issues/12989) to 20.6: Fix `Block structure mismatch` error for queries with UNION and JOIN'. [#13049](https://github.com/ClickHouse/ClickHouse/pull/13049) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12982](https://github.com/ClickHouse/ClickHouse/issues/12982) to 20.6: Merging [#12548](https://github.com/ClickHouse/ClickHouse/issues/12548) - Correction to `merge_with_ttl_timeout` logic by @excitoon'. [#13031](https://github.com/ClickHouse/ClickHouse/pull/13031) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12760](https://github.com/ClickHouse/ClickHouse/issues/12760) to 20.6: Sticking mutations bug'. [#13011](https://github.com/ClickHouse/ClickHouse/pull/13011) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12664](https://github.com/ClickHouse/ClickHouse/issues/12664) to 20.6: Fix move_functions_out_of_any optimisation with lambda'. [#12994](https://github.com/ClickHouse/ClickHouse/pull/12994) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12658](https://github.com/ClickHouse/ClickHouse/issues/12658) to 20.6: Fix SIGSEGV in StorageKafka when broker is unavailable'. [#12971](https://github.com/ClickHouse/ClickHouse/pull/12971) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12663](https://github.com/ClickHouse/ClickHouse/issues/12663) to 20.6: ISSUES-12293 allow push predicate when subquery contains with clause'. [#12868](https://github.com/ClickHouse/ClickHouse/pull/12868) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12672](https://github.com/ClickHouse/ClickHouse/issues/12672) to 20.6: Fix memory tracking for input_format_parallel_parsing'. [#12864](https://github.com/ClickHouse/ClickHouse/pull/12864) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12492](https://github.com/ClickHouse/ClickHouse/issues/12492) to 20.6: Make read buffer size lower, while reading from compact parts'. [#12862](https://github.com/ClickHouse/ClickHouse/pull/12862) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12659](https://github.com/ClickHouse/ClickHouse/issues/12659) to 20.6: ISSUES-10572 fix bloom filter with const column'. [#12858](https://github.com/ClickHouse/ClickHouse/pull/12858) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12480](https://github.com/ClickHouse/ClickHouse/issues/12480) to 20.6: [RFC] Fix SETTINGS parse after FORMAT'. [#12803](https://github.com/ClickHouse/ClickHouse/pull/12803) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12458](https://github.com/ClickHouse/ClickHouse/issues/12458) to 20.6: Fix crash in 'JOIN dict ON expr(dict_key) = k''. [#12725](https://github.com/ClickHouse/ClickHouse/pull/12725) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12302](https://github.com/ClickHouse/ClickHouse/issues/12302) to 20.6: kafka: fix SIGSEGV if there is a message with error in the middle of the batch'. [#12724](https://github.com/ClickHouse/ClickHouse/pull/12724) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12382](https://github.com/ClickHouse/ClickHouse/issues/12382) to 20.6: Better errors for CLEAR/DROP columns (possibly in partitions)'. [#12723](https://github.com/ClickHouse/ClickHouse/pull/12723) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12519](https://github.com/ClickHouse/ClickHouse/issues/12519) to 20.6: Fixing race condition in live view tables which could cause data duplication and live view tests'. [#12722](https://github.com/ClickHouse/ClickHouse/pull/12722) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12545](https://github.com/ClickHouse/ClickHouse/issues/12545) to 20.6: Fix loading parts without checksums'. [#12721](https://github.com/ClickHouse/ClickHouse/pull/12721) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12624](https://github.com/ClickHouse/ClickHouse/issues/12624) to 20.6: Fix error message about granularity'. [#12720](https://github.com/ClickHouse/ClickHouse/pull/12720) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12618](https://github.com/ClickHouse/ClickHouse/issues/12618) to 20.6: Fix external sort pipeline stuck'. [#12699](https://github.com/ClickHouse/ClickHouse/pull/12699) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12633](https://github.com/ClickHouse/ClickHouse/issues/12633) to 20.6: Fix "There is no supertype" error on ALTER UPDATE [#7306](https://github.com/ClickHouse/ClickHouse/issues/7306)'. [#12698](https://github.com/ClickHouse/ClickHouse/pull/12698) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12648](https://github.com/ClickHouse/ClickHouse/issues/12648) to 20.6: Add support for function if with Array(UUID) arguments'. [#12697](https://github.com/ClickHouse/ClickHouse/pull/12697) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12625](https://github.com/ClickHouse/ClickHouse/issues/12625) to 20.6: Better exception during directories creation'. [#12696](https://github.com/ClickHouse/ClickHouse/pull/12696) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* NO CL ENTRY: 'Backport [#12613](https://github.com/ClickHouse/ClickHouse/issues/12613) to 20.6: Remove sort description from streams'. [#12695](https://github.com/ClickHouse/ClickHouse/pull/12695) ([robot-clickhouse](https://github.com/robot-clickhouse)). - #### NO CL CATEGORY * ... [#12431](https://github.com/ClickHouse/ClickHouse/pull/12431) ([Tom Bombadil](https://github.com/ithangzhou)). * * Not for changelog. [#12265](https://github.com/ClickHouse/ClickHouse/pull/12265) ([Alexander Kuzmenkov](https://github.com/akuzm)). - - ## ClickHouse release 20.5 ### ClickHouse release v20.5.2.7-stable 2020-07-02 From 95f95b1f3782521f075e13c4f1170bc46d83ff9b Mon Sep 17 00:00:00 2001 From: vladimir golovchenko Date: Fri, 7 Aug 2020 01:02:50 -0700 Subject: [PATCH 150/374] Fixed misspelling links. --- docs/en/development/developer-instruction.md | 4 ++-- docs/es/development/developer-instruction.md | 2 +- docs/fa/development/developer-instruction.md | 2 +- docs/fr/development/developer-instruction.md | 2 +- docs/ja/development/developer-instruction.md | 2 +- docs/ru/development/developer-instruction.md | 2 +- docs/tr/development/developer-instruction.md | 2 +- docs/zh/development/developer-instruction.md | 2 +- 8 files changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index 7e6a0fd6a17..abe32ee8bc6 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -7,7 +7,7 @@ toc_title: For Beginners Building of ClickHouse is supported on Linux, FreeBSD and Mac OS X. -If you use Windows, you need to create a virtual machine with Ubuntu. To start working with a virtual machine please install VirtualBox. You can download Ubuntu from the website: https://www.ubuntu.com/\#download. Please create a virtual machine from the downloaded image (you should reserve at least 4GB of RAM for it). To run a command-line terminal in Ubuntu, please locate a program containing the word “terminal” in its name (gnome-terminal, konsole etc.) or just press Ctrl+Alt+T. +If you use Windows, you need to create a virtual machine with Ubuntu. To start working with a virtual machine please install VirtualBox. You can download Ubuntu from the website: https://www.ubuntu.com/#download. Please create a virtual machine from the downloaded image (you should reserve at least 4GB of RAM for it). To run a command-line terminal in Ubuntu, please locate a program containing the word “terminal” in its name (gnome-terminal, konsole etc.) or just press Ctrl+Alt+T. ClickHouse cannot work or build on a 32-bit system. You should acquire access to a 64-bit system and you can continue reading. @@ -137,7 +137,7 @@ Official Yandex builds currently use GCC because it generates machine code of sl To install GCC on Ubuntu run: `sudo apt install gcc g++` -Check the version of gcc: `gcc --version`. If it is below 9, then follow the instruction here: https://clickhouse.tech/docs/en/development/build/\#install-gcc-9. +Check the version of gcc: `gcc --version`. If it is below 9, then follow the instruction here: https://clickhouse.tech/docs/en/development/build/#install-gcc-9. Mac OS X build is supported only for Clang. Just run `brew install llvm` diff --git a/docs/es/development/developer-instruction.md b/docs/es/development/developer-instruction.md index 8f21e851b89..e796bcb5473 100644 --- a/docs/es/development/developer-instruction.md +++ b/docs/es/development/developer-instruction.md @@ -9,7 +9,7 @@ La construcción de ClickHouse es compatible con Linux, FreeBSD y Mac OS X. # Si utiliza Windows {#if-you-use-windows} -Si usa Windows, necesita crear una máquina virtual con Ubuntu. Para comenzar a trabajar con una máquina virtual, instale VirtualBox. Puede descargar Ubuntu desde el sitio web: https://www.ubuntu.com/\#download. Por favor, cree una máquina virtual a partir de la imagen descargada (debe reservar al menos 4 GB de RAM para ello). Para ejecutar un terminal de línea de comandos en Ubuntu, busque un programa que contenga la palabra “terminal” en su nombre (gnome-terminal, konsole etc.) o simplemente presione Ctrl + Alt + T. +Si usa Windows, necesita crear una máquina virtual con Ubuntu. Para comenzar a trabajar con una máquina virtual, instale VirtualBox. Puede descargar Ubuntu desde el sitio web: https://www.ubuntu.com/#download. Por favor, cree una máquina virtual a partir de la imagen descargada (debe reservar al menos 4 GB de RAM para ello). Para ejecutar un terminal de línea de comandos en Ubuntu, busque un programa que contenga la palabra “terminal” en su nombre (gnome-terminal, konsole etc.) o simplemente presione Ctrl + Alt + T. # Si utiliza un sistema de 32 bits {#if-you-use-a-32-bit-system} diff --git a/docs/fa/development/developer-instruction.md b/docs/fa/development/developer-instruction.md index 287960aff04..7072d84da1a 100644 --- a/docs/fa/development/developer-instruction.md +++ b/docs/fa/development/developer-instruction.md @@ -11,7 +11,7 @@ toc_title: "\u062F\u0633\u062A\u0648\u0631\u0627\u0644\u0639\u0645\u0644 \u062A\ # در صورت استفاده از ویندوز {#if-you-use-windows} -اگر شما استفاده از ویندوز, شما نیاز به ایجاد یک ماشین مجازی با اوبونتو. برای شروع کار با یک ماشین مجازی لطفا مجازی نصب کنید. شما می توانید اوبونتو را از وب سایت دانلود کنید: https://www.ubuntu.com/\#download.لطفا یک ماشین مجازی از تصویر دانلود شده ایجاد کنید (شما باید حداقل 4 گیگابایت رم را رزرو کنید). برای اجرای یک ترمینال خط فرمان در اوبونتو, لطفا یک برنامه حاوی کلمه قرار “terminal” به نام (گنوم ترمینال, کنسول و غیره.) یا فقط کنترل را فشار دهید +اگر شما استفاده از ویندوز, شما نیاز به ایجاد یک ماشین مجازی با اوبونتو. برای شروع کار با یک ماشین مجازی لطفا مجازی نصب کنید. شما می توانید اوبونتو را از وب سایت دانلود کنید: https://www.ubuntu.com/#download.لطفا یک ماشین مجازی از تصویر دانلود شده ایجاد کنید (شما باید حداقل 4 گیگابایت رم را رزرو کنید). برای اجرای یک ترمینال خط فرمان در اوبونتو, لطفا یک برنامه حاوی کلمه قرار “terminal” به نام (گنوم ترمینال, کنسول و غیره.) یا فقط کنترل را فشار دهید # اگر از یک سیستم 32 بیتی استفاده می کنید {#if-you-use-a-32-bit-system} diff --git a/docs/fr/development/developer-instruction.md b/docs/fr/development/developer-instruction.md index a20066fa3f7..de72903af2d 100644 --- a/docs/fr/development/developer-instruction.md +++ b/docs/fr/development/developer-instruction.md @@ -9,7 +9,7 @@ La construction de ClickHouse est prise en charge sous Linux, FreeBSD et Mac OS # Si Vous Utilisez Windows {#if-you-use-windows} -Si vous utilisez Windows, vous devez créer une machine virtuelle avec Ubuntu. Pour commencer à travailler avec une machine virtuelle, installez VirtualBox. Vous pouvez télécharger Ubuntu sur le site: https://www.ubuntu.com/\#download. veuillez créer une machine virtuelle à partir de l'image téléchargée (vous devez réserver au moins 4 Go de RAM pour cela). Pour exécuter un terminal de ligne de commande dans Ubuntu, recherchez un programme contenant le mot “terminal” dans son nom (gnome-terminal, konsole etc.) ou appuyez simplement sur Ctrl + Alt + T. +Si vous utilisez Windows, vous devez créer une machine virtuelle avec Ubuntu. Pour commencer à travailler avec une machine virtuelle, installez VirtualBox. Vous pouvez télécharger Ubuntu sur le site: https://www.ubuntu.com/#download. veuillez créer une machine virtuelle à partir de l'image téléchargée (vous devez réserver au moins 4 Go de RAM pour cela). Pour exécuter un terminal de ligne de commande dans Ubuntu, recherchez un programme contenant le mot “terminal” dans son nom (gnome-terminal, konsole etc.) ou appuyez simplement sur Ctrl + Alt + T. # Si vous utilisez un système 32 bits {#if-you-use-a-32-bit-system} diff --git a/docs/ja/development/developer-instruction.md b/docs/ja/development/developer-instruction.md index 6441e77185f..cacac32ca5f 100644 --- a/docs/ja/development/developer-instruction.md +++ b/docs/ja/development/developer-instruction.md @@ -9,7 +9,7 @@ ClickHouseのビルドはLinux、FreeBSD、Mac OS Xでサポートされてい # Windowsを使用する場合 {#if-you-use-windows} -Windowsを使用する場合は、Ubuntuで仮想マシンを作成する必要があります。 するのは、仮想マシンをインストールしてくださいVirtualBox. ダウンロードできますUbuntuのウェブサイト:https://www.ubuntu.com/\#download. を作成してください仮想マシンからダウンロードした画像を保少なくとも4GB RAMめます。 Ubuntuでコマンドライン端末を実行するには、その単語を含むプログラムを見つけてください “terminal” その名前で(gnome-terminal、konsoleなど。)または単にCtrl+Alt+Tを押します。 +Windowsを使用する場合は、Ubuntuで仮想マシンを作成する必要があります。 するのは、仮想マシンをインストールしてくださいVirtualBox. ダウンロードできますUbuntuのウェブサイト:https://www.ubuntu.com/#download. を作成してください仮想マシンからダウンロードした画像を保少なくとも4GB RAMめます。 Ubuntuでコマンドライン端末を実行するには、その単語を含むプログラムを見つけてください “terminal” その名前で(gnome-terminal、konsoleなど。)または単にCtrl+Alt+Tを押します。 # 32ビットシステムを使用する場合 {#if-you-use-a-32-bit-system} diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 775b156dff5..431b3f272a7 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -2,7 +2,7 @@ # Если вы используете Windows {#esli-vy-ispolzuete-windows} -Если вы используете Windows, вам потребуется создать виртуальную машину с Ubuntu. Для работы с виртуальной машиной, установите VirtualBox. Скачать Ubuntu можно на сайте: https://www.ubuntu.com/\#download Создайте виртуальную машину из полученного образа. Выделите для неё не менее 4 GB оперативной памяти. Для запуска терминала в Ubuntu, найдите в меню программу со словом terminal (gnome-terminal, konsole или что-то в этом роде) или нажмите Ctrl+Alt+T. +Если вы используете Windows, вам потребуется создать виртуальную машину с Ubuntu. Для работы с виртуальной машиной, установите VirtualBox. Скачать Ubuntu можно на сайте: https://www.ubuntu.com/#download Создайте виртуальную машину из полученного образа. Выделите для неё не менее 4 GB оперативной памяти. Для запуска терминала в Ubuntu, найдите в меню программу со словом terminal (gnome-terminal, konsole или что-то в этом роде) или нажмите Ctrl+Alt+T. # Если вы используете 32-битную систему {#esli-vy-ispolzuete-32-bitnuiu-sistemu} diff --git a/docs/tr/development/developer-instruction.md b/docs/tr/development/developer-instruction.md index 0ca5f9cdd63..648d164e7b2 100644 --- a/docs/tr/development/developer-instruction.md +++ b/docs/tr/development/developer-instruction.md @@ -9,7 +9,7 @@ ClickHouse binası Linux, FreeBSD ve Mac OS X üzerinde desteklenmektedir. # Windows Kullanıyorsanız {#if-you-use-windows} -Windows kullanıyorsanız, Ubuntu ile bir sanal makine oluşturmanız gerekir. Bir sanal makine ile çalışmaya başlamak için VirtualBox yükleyin. UB :unt :u'yu web sitesinden indirebilirsiniz: https://www.ubuntu.com/\#download. lütfen indirilen görüntüden bir sanal makine oluşturun (bunun için en az 4GB RAM ayırmalısınız). Ubuntu'da bir komut satırı terminali çalıştırmak için lütfen kelimeyi içeren bir program bulun “terminal” adına (gnome-terminal, konsole vb.)) veya sadece Ctrl+Alt+T tuşlarına basın. +Windows kullanıyorsanız, Ubuntu ile bir sanal makine oluşturmanız gerekir. Bir sanal makine ile çalışmaya başlamak için VirtualBox yükleyin. UB :unt :u'yu web sitesinden indirebilirsiniz: https://www.ubuntu.com/#download. lütfen indirilen görüntüden bir sanal makine oluşturun (bunun için en az 4GB RAM ayırmalısınız). Ubuntu'da bir komut satırı terminali çalıştırmak için lütfen kelimeyi içeren bir program bulun “terminal” adına (gnome-terminal, konsole vb.)) veya sadece Ctrl+Alt+T tuşlarına basın. # 32 bit sistem kullanıyorsanız {#if-you-use-a-32-bit-system} diff --git a/docs/zh/development/developer-instruction.md b/docs/zh/development/developer-instruction.md index b40e6db3af1..ea86fd8b63a 100644 --- a/docs/zh/development/developer-instruction.md +++ b/docs/zh/development/developer-instruction.md @@ -2,7 +2,7 @@ ClickHose支持Linux,FreeBSD 及 Mac OS X 系统。 # Windows使用指引 {#windowsshi-yong-zhi-yin} -如果您的系统是Windows,则需要创建Ubuntu虚拟机。可以安装VirtualBox来构建虚拟机。Ubuntu的下载链接为:https://www.ubuntu.com/\#download 。请使用下载好的镜像创建一个虚拟机(请确保虚拟机有至少4GB的内存容量)。在Ubuntu中使用«terminal»程序(gnome-terminal,konsole等)运行命令行终端,或使用快捷键Ctrl+Alt+T。 +如果您的系统是Windows,则需要创建Ubuntu虚拟机。可以安装VirtualBox来构建虚拟机。Ubuntu的下载链接为:https://www.ubuntu.com/#download 。请使用下载好的镜像创建一个虚拟机(请确保虚拟机有至少4GB的内存容量)。在Ubuntu中使用«terminal»程序(gnome-terminal,konsole等)运行命令行终端,或使用快捷键Ctrl+Alt+T。 # 在GitHub上创建源码库 {#zai-githubshang-chuang-jian-yuan-ma-ku} From 987b4207090668fc24a3f7efb08b8dcc3fda31a1 Mon Sep 17 00:00:00 2001 From: vladimir golovchenko Date: Fri, 7 Aug 2020 01:04:20 -0700 Subject: [PATCH 151/374] Fixed the wrong link to Git Cheat Sheet. --- docs/en/development/developer-instruction.md | 2 +- docs/es/development/developer-instruction.md | 2 +- docs/fa/development/developer-instruction.md | 2 +- docs/fr/development/developer-instruction.md | 2 +- docs/ja/development/developer-instruction.md | 2 +- docs/ru/development/developer-instruction.md | 2 +- docs/tr/development/developer-instruction.md | 2 +- docs/zh/development/developer-instruction.md | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index abe32ee8bc6..bca63be7480 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -28,7 +28,7 @@ To do that in Ubuntu you would run in the command line terminal: sudo apt update sudo apt install git -A brief manual on using Git can be found here: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf. +A brief manual on using Git can be found here: https://education.github.com/git-cheat-sheet-education.pdf. For a detailed manual on Git see https://git-scm.com/book/en/v2. ## Cloning a Repository to Your Development Machine {#cloning-a-repository-to-your-development-machine} diff --git a/docs/es/development/developer-instruction.md b/docs/es/development/developer-instruction.md index e796bcb5473..71f928ff73b 100644 --- a/docs/es/development/developer-instruction.md +++ b/docs/es/development/developer-instruction.md @@ -32,7 +32,7 @@ Para hacer eso en Ubuntu, ejecutaría en la terminal de línea de comandos: sudo apt update sudo apt install git -Puede encontrar un breve manual sobre el uso de Git aquí: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf . +Puede encontrar un breve manual sobre el uso de Git aquí: https://education.github.com/git-cheat-sheet-education.pdf . Para obtener un manual detallado sobre Git, consulte https://git-scm.com/book/en/v2 . # Clonación de un repositorio en su máquina de desarrollo {#cloning-a-repository-to-your-development-machine} diff --git a/docs/fa/development/developer-instruction.md b/docs/fa/development/developer-instruction.md index 7072d84da1a..f6446517d34 100644 --- a/docs/fa/development/developer-instruction.md +++ b/docs/fa/development/developer-instruction.md @@ -34,7 +34,7 @@ toc_title: "\u062F\u0633\u062A\u0648\u0631\u0627\u0644\u0639\u0645\u0644 \u062A\ sudo apt update sudo apt install git -کتابچه راهنمای مختصر در استفاده از دستگاه گوارش را می توان یافت: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf. +کتابچه راهنمای مختصر در استفاده از دستگاه گوارش را می توان یافت: https://education.github.com/git-cheat-sheet-education.pdf. برای یک کتابچه راهنمای دقیق در دستگاه گوارش را ببینید https://git-scm.com/book/en/v2. # شبیه سازی یک مخزن به دستگاه توسعه خود را {#cloning-a-repository-to-your-development-machine} diff --git a/docs/fr/development/developer-instruction.md b/docs/fr/development/developer-instruction.md index de72903af2d..2528e279cbd 100644 --- a/docs/fr/development/developer-instruction.md +++ b/docs/fr/development/developer-instruction.md @@ -32,7 +32,7 @@ Pour ce faire dans Ubuntu vous exécutez dans le terminal de ligne de commande: sudo apt update sudo apt install git -Un bref manuel sur l'utilisation de Git peut être trouvé ici: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf. +Un bref manuel sur l'utilisation de Git peut être trouvé ici: https://education.github.com/git-cheat-sheet-education.pdf. Pour un manuel détaillé sur Git voir https://git-scm.com/book/en/v2. # Clonage D'un référentiel sur votre machine de développement {#cloning-a-repository-to-your-development-machine} diff --git a/docs/ja/development/developer-instruction.md b/docs/ja/development/developer-instruction.md index cacac32ca5f..a8b95168fba 100644 --- a/docs/ja/development/developer-instruction.md +++ b/docs/ja/development/developer-instruction.md @@ -32,7 +32,7 @@ Ubuntuでこれを行うには、コマンドラインターミナルで実行 sudo apt update sudo apt install git -簡単なマニュアルを使用Gitで、できるだけ早く送ってくださhttps://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf. +簡単なマニュアルを使用Gitで、できるだけ早く送ってくださhttps://education.github.com/git-cheat-sheet-education.pdf. 詳細なマニュアルGit見https://git-scm.com/book/en/v2. # 開発マシンへのリポジトリの複製 {#cloning-a-repository-to-your-development-machine} diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 431b3f272a7..0bb09621779 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -23,7 +23,7 @@ ClickHouse не работает и не собирается на 32-битны sudo apt update sudo apt install git -Краткое руководство по использованию Git: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf +Краткое руководство по использованию Git: https://education.github.com/git-cheat-sheet-education.pdf Подробное руководство по использованию Git: https://git-scm.com/book/ru/v2 diff --git a/docs/tr/development/developer-instruction.md b/docs/tr/development/developer-instruction.md index 648d164e7b2..619aa60ee22 100644 --- a/docs/tr/development/developer-instruction.md +++ b/docs/tr/development/developer-instruction.md @@ -32,7 +32,7 @@ Bunu Ubuntu'da yapmak için komut satırı terminalinde çalışırsınız: sudo apt update sudo apt install git -Git kullanımı ile ilgili kısa bir el kitabı burada bulunabilir: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf. +Git kullanımı ile ilgili kısa bir el kitabı burada bulunabilir: https://education.github.com/git-cheat-sheet-education.pdf. Git ile ilgili ayrıntılı bir el kitabı için bkz. https://git-scm.com/book/en/v2. # Geliştirme Makinenize bir depo klonlama {#cloning-a-repository-to-your-development-machine} diff --git a/docs/zh/development/developer-instruction.md b/docs/zh/development/developer-instruction.md index ea86fd8b63a..a0d7b5baefd 100644 --- a/docs/zh/development/developer-instruction.md +++ b/docs/zh/development/developer-instruction.md @@ -21,7 +21,7 @@ ClickHose支持Linux,FreeBSD 及 Mac OS X 系统。 sudo apt update sudo apt install git -在https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf中找到有关使用Git的简易手册。有关Git的详细手册,请参见: https://git-scm.com/book/ru/v2 。 +在https://education.github.com/git-cheat-sheet-education.pdf中找到有关使用Git的简易手册。有关Git的详细手册,请参见: https://git-scm.com/book/ru/v2 。 # 拷贝源码库到开发机 {#kao-bei-yuan-ma-ku-dao-kai-fa-ji} From 568b8d31b4c5628cc1ac0aa7a943b7632b15ee12 Mon Sep 17 00:00:00 2001 From: vladimir golovchenko Date: Fri, 7 Aug 2020 01:06:58 -0700 Subject: [PATCH 152/374] Fixed the link to a list of tasks: it points to open issues marked as 'easy'. --- docs/en/development/developer-instruction.md | 2 +- docs/es/development/developer-instruction.md | 2 +- docs/fa/development/developer-instruction.md | 2 +- docs/fr/development/developer-instruction.md | 2 +- docs/ja/development/developer-instruction.md | 2 +- docs/ru/development/developer-instruction.md | 2 +- docs/tr/development/developer-instruction.md | 2 +- docs/zh/development/developer-instruction.md | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index bca63be7480..d1707951501 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -245,7 +245,7 @@ The Code Style Guide: https://clickhouse.tech/docs/en/development/style/ Writing tests: https://clickhouse.tech/docs/en/development/tests/ -List of tasks: https://github.com/ClickHouse/ClickHouse/contribute +List of tasks: https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aopen+is%3Aissue+label%3A%22easy+task%22 ## Test Data {#test-data} diff --git a/docs/es/development/developer-instruction.md b/docs/es/development/developer-instruction.md index 71f928ff73b..20a1bb1d528 100644 --- a/docs/es/development/developer-instruction.md +++ b/docs/es/development/developer-instruction.md @@ -249,7 +249,7 @@ La Guía de estilo de código: https://clickhouse.tech/docs/en/development/style Pruebas de escritura: https://clickhouse.tech/docs/en/development/tests/ -Lista de tareas: https://github.com/ClickHouse/ClickHouse/contribute +Lista de tareas: https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aopen+is%3Aissue+label%3A%22easy+task%22 # Datos de prueba {#test-data} diff --git a/docs/fa/development/developer-instruction.md b/docs/fa/development/developer-instruction.md index f6446517d34..ce87a189489 100644 --- a/docs/fa/development/developer-instruction.md +++ b/docs/fa/development/developer-instruction.md @@ -251,7 +251,7 @@ KDevelop و QTCreator دیگر از جایگزین های بسیار خوبی ا تست نوشتن: https://clickhouse.فناوری / اسناد/توسعه/تست/ -فهرست تکلیفها: https://github.com/ClickHouse/ClickHouse/contribute +فهرست تکلیفها: https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aopen+is%3Aissue+label%3A%22easy+task%22 # داده های تست {#test-data} diff --git a/docs/fr/development/developer-instruction.md b/docs/fr/development/developer-instruction.md index 2528e279cbd..847e3960bc7 100644 --- a/docs/fr/development/developer-instruction.md +++ b/docs/fr/development/developer-instruction.md @@ -249,7 +249,7 @@ Le code Style Guide: https://clickhouse.tech/docs/fr/développement/style/ Rédaction de tests: https://clickhouse.tech/docs/fr/développement/tests/ -Liste des tâches: https://github.com/ClickHouse/ClickHouse/contribute +Liste des tâches: https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aopen+is%3Aissue+label%3A%22easy+task%22 # Des Données De Test {#test-data} diff --git a/docs/ja/development/developer-instruction.md b/docs/ja/development/developer-instruction.md index a8b95168fba..e3b7e60a7cd 100644 --- a/docs/ja/development/developer-instruction.md +++ b/docs/ja/development/developer-instruction.md @@ -249,7 +249,7 @@ KDevelopとQTCreatorは、ClickHouseを開発するためのIDEの他の優れ 筆記試験:https://clickhouse.tech/docs/en/development/tests/ -タスクのリスト:https://github.com/ClickHouse/ClickHouse/contribute +タスクのリスト:https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aopen+is%3Aissue+label%3A%22easy+task%22 # テストデータ {#test-data} diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 0bb09621779..251bf6bc3bf 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -244,7 +244,7 @@ Mac OS X: Разработка тестов: https://clickhouse.tech/docs/ru/development/tests/ -Список задач: https://github.com/ClickHouse/ClickHouse/contribute +Список задач: https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aopen+is%3Aissue+label%3A%22easy+task%22 # Тестовые данные {#testovye-dannye} diff --git a/docs/tr/development/developer-instruction.md b/docs/tr/development/developer-instruction.md index 619aa60ee22..13a1efdd156 100644 --- a/docs/tr/development/developer-instruction.md +++ b/docs/tr/development/developer-instruction.md @@ -249,7 +249,7 @@ Kod stili Kılavuzu: https://clickhouse.tech / doscs / TR / development / style/ Yazma testleri: https://clickhouse.teknoloji / doscs / TR / geliştirme / testler/ -Görevlerin listesi: https://github.com/ClickHouse/ClickHouse/contribute +Görevlerin listesi: https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aopen+is%3Aissue+label%3A%22easy+task%22 # Test Verileri {#test-data} diff --git a/docs/zh/development/developer-instruction.md b/docs/zh/development/developer-instruction.md index a0d7b5baefd..ac5fb0c7ca2 100644 --- a/docs/zh/development/developer-instruction.md +++ b/docs/zh/development/developer-instruction.md @@ -234,7 +234,7 @@ ClickHouse的架构描述可以在此处查看:https://clickhouse.tech/docs/en 编写测试用例:https://clickhouse.tech/docs/en/development/tests/ -任务列表:https://github.com/ClickHouse/ClickHouse/contribute +任务列表:https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aopen+is%3Aissue+label%3A%22easy+task%22 # 测试数据 {#ce-shi-shu-ju} From 057a596aa29627f5f5a27114f56a5fc489eeec99 Mon Sep 17 00:00:00 2001 From: vladimir golovchenko Date: Fri, 7 Aug 2020 01:18:38 -0700 Subject: [PATCH 153/374] Fixed the wrong command to run clickhouse-client. --- docs/en/development/developer-instruction.md | 2 +- docs/es/development/developer-instruction.md | 2 +- docs/fa/development/developer-instruction.md | 2 +- docs/fr/development/developer-instruction.md | 2 +- docs/ja/development/developer-instruction.md | 2 +- docs/ru/development/developer-instruction.md | 2 +- docs/tr/development/developer-instruction.md | 2 +- docs/zh/development/developer-instruction.md | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index d1707951501..db884649c2d 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -208,7 +208,7 @@ To run the server under the current user you need to navigate to `ClickHouse/pro In this case, ClickHouse will use config files located in the current directory. You can run `clickhouse server` from any directory specifying the path to a config file as a command-line parameter `--config-file`. -To connect to ClickHouse with clickhouse-client in another terminal navigate to `ClickHouse/build/programs/` and run `clickhouse client`. +To connect to ClickHouse with clickhouse-client in another terminal navigate to `ClickHouse/build/programs/` and run `./clickhouse client`. If you get `Connection refused` message on Mac OS X or FreeBSD, try specifying host address 127.0.0.1: diff --git a/docs/es/development/developer-instruction.md b/docs/es/development/developer-instruction.md index 20a1bb1d528..8e076a60ac4 100644 --- a/docs/es/development/developer-instruction.md +++ b/docs/es/development/developer-instruction.md @@ -212,7 +212,7 @@ Para ejecutar el servidor bajo el usuario actual, debe navegar hasta `ClickHouse En este caso, ClickHouse usará archivos de configuración ubicados en el directorio actual. Puede ejecutar `clickhouse server` desde cualquier directorio que especifique la ruta a un archivo de configuración como un parámetro de línea de comandos `--config-file`. -Para conectarse a ClickHouse con clickhouse-client en otro terminal, vaya a `ClickHouse/build/programs/` y ejecutar `clickhouse client`. +Para conectarse a ClickHouse con clickhouse-client en otro terminal, vaya a `ClickHouse/build/programs/` y ejecutar `./clickhouse client`. Si usted consigue `Connection refused` mensaje en Mac OS X o FreeBSD, intente especificar la dirección de host 127.0.0.1: diff --git a/docs/fa/development/developer-instruction.md b/docs/fa/development/developer-instruction.md index ce87a189489..94e5535d67f 100644 --- a/docs/fa/development/developer-instruction.md +++ b/docs/fa/development/developer-instruction.md @@ -214,7 +214,7 @@ toc_title: "\u062F\u0633\u062A\u0648\u0631\u0627\u0644\u0639\u0645\u0644 \u062A\ در این مورد, تاتر خواهد فایل های پیکربندی واقع در دایرکتوری جاری استفاده. شما می توانید اجرا کنید `clickhouse server` از هر دایرکتوری مشخص کردن مسیر به یک فایل پیکربندی به عنوان یک پارامتر خط فرمان `--config-file`. -برای اتصال به ClickHouse با clickhouse-مشتری در یکی دیگر از ترمینال حرکت به `ClickHouse/build/programs/` و فرار کن `clickhouse client`. +برای اتصال به ClickHouse با clickhouse-مشتری در یکی دیگر از ترمینال حرکت به `ClickHouse/build/programs/` و فرار کن `./clickhouse client`. اگر شما `Connection refused` سعی کنید مشخص نشانی میزبان 127.0.0.1: diff --git a/docs/fr/development/developer-instruction.md b/docs/fr/development/developer-instruction.md index 847e3960bc7..8d549d3a670 100644 --- a/docs/fr/development/developer-instruction.md +++ b/docs/fr/development/developer-instruction.md @@ -212,7 +212,7 @@ Pour exécuter le serveur sous l'utilisateur actuel vous devez naviguer vers `Cl Dans ce cas, ClickHouse utilisera les fichiers de configuration situés dans le répertoire courant. Vous pouvez l'exécuter `clickhouse server` depuis n'importe quel répertoire spécifiant le chemin d'accès à un fichier de configuration en tant que paramètre de ligne de commande `--config-file`. -Pour vous connecter à ClickHouse avec clickhouse-client dans un autre terminal, accédez à `ClickHouse/build/programs/` et exécuter `clickhouse client`. +Pour vous connecter à ClickHouse avec clickhouse-client dans un autre terminal, accédez à `ClickHouse/build/programs/` et exécuter `./clickhouse client`. Si vous obtenez `Connection refused` message sur Mac OS X ou FreeBSD, essayez de spécifier l'adresse hôte 127.0.0.1: diff --git a/docs/ja/development/developer-instruction.md b/docs/ja/development/developer-instruction.md index e3b7e60a7cd..6c7e1474dab 100644 --- a/docs/ja/development/developer-instruction.md +++ b/docs/ja/development/developer-instruction.md @@ -212,7 +212,7 @@ Mac OS X: この場合、ClickHouseは現在のディレクトリにある設定ファイルを使用します。 実行できます `clickhouse server` からのディレクトリのパスを指定し、設定ファイルとしてコマンドラインパラメータ `--config-file`. -別のターミナルでclickhouse-clientを使用してClickHouseに接続するには、次の場所に移動します `ClickHouse/build/programs/` と実行 `clickhouse client`. +別のターミナルでclickhouse-clientを使用してClickHouseに接続するには、次の場所に移動します `ClickHouse/build/programs/` と実行 `./clickhouse client`. あなたが得る場合 `Connection refused` メッセージMac OS XまたはFreeBSDでは、ホストアドレス127.0.0.1を指定してみます: diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 251bf6bc3bf..025772ea3ca 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -207,7 +207,7 @@ Mac OS X: В этом случае, ClickHouse будет использовать конфигурационные файлы, расположенные в текущей директории. Вы можете запустить `clickhouse server` из любой директории, передав ему путь к конфигурационному файлу в аргументе командной строки `--config-file`. -Для подключения к ClickHouse с помощью clickhouse-client, в соседнем терминале, зайдите в директорию `ClickHouse/build/programs/` и выполните `clickhouse client`. +Для подключения к ClickHouse с помощью clickhouse-client, в соседнем терминале, зайдите в директорию `ClickHouse/build/programs/` и выполните `./clickhouse client`. Если вы получили сообщение `Connection refused` на Mac OS X или FreeBSD, то укажите для клиента 127.0.0.1 в качестве имени хоста: diff --git a/docs/tr/development/developer-instruction.md b/docs/tr/development/developer-instruction.md index 13a1efdd156..284989c652d 100644 --- a/docs/tr/development/developer-instruction.md +++ b/docs/tr/development/developer-instruction.md @@ -212,7 +212,7 @@ Sunucuyu geçerli kullanıcı altında çalıştırmak için aşağıdakilere gi Bu durumda, ClickHouse geçerli dizinde bulunan yapılandırma dosyalarını kullanır. Koş youabilirsiniz `clickhouse server` komut satırı parametresi olarak bir yapılandırma dosyasının yolunu belirten herhangi bir dizinden `--config-file`. -Başka bir terminalde clickhouse-client ile Clickhouse'a bağlanmak için `ClickHouse/build/programs/` ve koş `clickhouse client`. +Başka bir terminalde clickhouse-client ile Clickhouse'a bağlanmak için `ClickHouse/build/programs/` ve koş `./clickhouse client`. Eğer alırsanız `Connection refused` Mac OS X veya Freebsd'de mesaj, ana bilgisayar adresi 127.0.0.1 belirtmeyi deneyin: diff --git a/docs/zh/development/developer-instruction.md b/docs/zh/development/developer-instruction.md index ac5fb0c7ca2..35bac0322bb 100644 --- a/docs/zh/development/developer-instruction.md +++ b/docs/zh/development/developer-instruction.md @@ -197,7 +197,7 @@ Yandex官方当前使用GCC构建ClickHouse,因为它生成的机器代码性 在这种情况下,ClickHouse将使用位于当前目录中的配置文件。您可以从任何目录运行`Clickhouse server`,并将配置文件`--config-file`的路径指定为命令行参数。 -在另外一个终端上连接ClickHouse的clickhouse-client客户端,请进入到`ClickHouse/build/programs/` 并运行`clickhouse client`。 +在另外一个终端上连接ClickHouse的clickhouse-client客户端,请进入到`ClickHouse/build/programs/` 并运行`./clickhouse client`。 如果您在Mac OS X 或者 FreeBSD上收到`Connection refused`的消息,请尝试指定主机地址为127.0.0.1: From 336a6155d8512d2d0d567469118aad2779674374 Mon Sep 17 00:00:00 2001 From: vladimir golovchenko Date: Fri, 7 Aug 2020 02:50:46 -0700 Subject: [PATCH 154/374] Actualized the versions of compilers and Standard Library. --- docs/en/development/style.md | 6 +++--- docs/es/development/style.md | 6 +++--- docs/fa/development/style.md | 6 +++--- docs/fr/development/style.md | 6 +++--- docs/ja/development/style.md | 6 +++--- docs/ru/development/style.md | 6 +++--- docs/tr/development/style.md | 6 +++--- docs/zh/development/style.md | 6 +++--- 8 files changed, 24 insertions(+), 24 deletions(-) diff --git a/docs/en/development/style.md b/docs/en/development/style.md index c239460b057..d1a066343cf 100644 --- a/docs/en/development/style.md +++ b/docs/en/development/style.md @@ -699,9 +699,9 @@ auto s = std::string{"Hello"}; But other things being equal, cross-platform or portable code is preferred. -**2.** Language: C++17. +**2.** Language: C++20 (see the list of available [C++20 features](https://en.cppreference.com/w/cpp/compiler_support#C.2B.2B20_features)). -**3.** Compiler: `gcc`. At this time (December 2017), the code is compiled using version 7.2. (It can also be compiled using `clang 4`.) +**3.** Compiler: `gcc`. At this time (August 2020), the code is compiled using version 9.3. (It can also be compiled using `clang 8`.) The standard library is used (`libstdc++` or `libc++`). @@ -747,7 +747,7 @@ If your code in the `master` branch is not buildable yet, exclude it from the bu ## Libraries {#libraries} -**1.** The C++14 standard library is used (experimental extensions are allowed), as well as `boost` and `Poco` frameworks. +**1.** The C++20 standard library is used (experimental extensions are allowed), as well as `boost` and `Poco` frameworks. **2.** If necessary, you can use any well-known libraries available in the OS package. diff --git a/docs/es/development/style.md b/docs/es/development/style.md index 7bf3dfc02d6..7e69423cf67 100644 --- a/docs/es/development/style.md +++ b/docs/es/development/style.md @@ -701,9 +701,9 @@ auto s = std::string{"Hello"}; Pero en igualdad de condiciones, se prefiere el código multiplataforma o portátil. -**2.** Idioma: C ++ 17. +**2.** Idioma: C++20. -**3.** Compilación: `gcc`. En este momento (diciembre de 2017), el código se compila utilizando la versión 7.2. (También se puede compilar usando `clang 4`.) +**3.** Compilación: `gcc`. En este momento (agosto 2020), el código se compila utilizando la versión 9.3. (También se puede compilar usando `clang 8`.) Se utiliza la biblioteca estándar (`libstdc++` o `libc++`). @@ -749,7 +749,7 @@ Si su código en el `master` branch todavía no se puede construir, excluirlo de ## Biblioteca {#libraries} -**1.** Se utiliza la biblioteca estándar de C ++ 14 (se permiten extensiones experimentales), así como `boost` y `Poco` marco. +**1.** Se utiliza la biblioteca estándar de C++20 (se permiten extensiones experimentales), así como `boost` y `Poco` marco. **2.** Si es necesario, puede usar cualquier biblioteca conocida disponible en el paquete del sistema operativo. diff --git a/docs/fa/development/style.md b/docs/fa/development/style.md index fb95b48d534..094d666e716 100644 --- a/docs/fa/development/style.md +++ b/docs/fa/development/style.md @@ -702,9 +702,9 @@ auto s = std::string{"Hello"}; اما چیزهای دیگر برابر بودن, کراس پلت فرم و یا کد قابل حمل ترجیح داده می شود. -**2.** زبان: ج++17. +**2.** زبان: ج++20. -**3.** کامپایلر: `gcc`. در این زمان (دسامبر 2017), کد با استفاده از نسخه وارد شده 7.2. (همچنین می تواند با استفاده از وارد شود `clang 4`.) +**3.** کامپایلر: `gcc`. در این زمان (اوت 2020), کد با استفاده از نسخه وارد شده 9.3. (همچنین می تواند با استفاده از وارد شود `clang 8`.) کتابخانه استاندارد استفاده شده است (`libstdc++` یا `libc++`). @@ -750,7 +750,7 @@ auto s = std::string{"Hello"}; ## کتابخانهها {#libraries} -**1.** ج++14 کتابخانه استاندارد استفاده شده است (پسوند تجربی مجاز), و همچنین `boost` و `Poco` چارچوب. +**1.** ج++20 کتابخانه استاندارد استفاده شده است (پسوند تجربی مجاز), و همچنین `boost` و `Poco` چارچوب. **2.** در صورت لزوم, شما می توانید هر کتابخانه شناخته شده موجود در بسته سیستم عامل استفاده. diff --git a/docs/fr/development/style.md b/docs/fr/development/style.md index ef50570fbdf..65cc1a507a9 100644 --- a/docs/fr/development/style.md +++ b/docs/fr/development/style.md @@ -701,9 +701,9 @@ auto s = std::string{"Hello"}; Mais toutes choses étant égales par ailleurs, le code multi-plateforme ou portable est préféré. -**2.** Langue: C++17. +**2.** Langue: C++20. -**3.** Compilateur: `gcc`. En ce moment (décembre 2017), le code est compilé en utilisant la version 7.2. (Il peut également être compilé en utilisant `clang 4`.) +**3.** Compilateur: `gcc`. En ce moment (août 2020), le code est compilé en utilisant la version 9.3. (Il peut également être compilé en utilisant `clang 8`.) La bibliothèque standard est utilisée (`libstdc++` ou `libc++`). @@ -749,7 +749,7 @@ Si votre code dans le `master` la branche n'est pas constructible pourtant, l'ex ## Bibliothèque {#libraries} -**1.** La bibliothèque standard C++14 est utilisée (les extensions expérimentales sont autorisées), ainsi que `boost` et `Poco` Framework. +**1.** La bibliothèque standard C++20 est utilisée (les extensions expérimentales sont autorisées), ainsi que `boost` et `Poco` Framework. **2.** Si nécessaire, vous pouvez utiliser toutes les bibliothèques bien connues disponibles dans le package OS. diff --git a/docs/ja/development/style.md b/docs/ja/development/style.md index ba2d744f6ef..21051c6cc12 100644 --- a/docs/ja/development/style.md +++ b/docs/ja/development/style.md @@ -701,9 +701,9 @@ auto s = std::string{"Hello"}; それが同じ場合には、クロス-プラットフォームまたは携帯コードが好ましい。 -**2.** 言語:C++17. +**2.** 言語:C++20. -**3.** コンパイラ: `gcc`. 2017年現在、コードはバージョン7.2を使用してコンパイルされている。 (以下を使ってコンパイルできます `clang 4`.) +**3.** コンパイラ: `gcc`. 2020年現在、コードはバージョン9.3を使用してコンパイルされている。 (以下を使ってコンパイルできます `clang 8`.) 標準ライブラリが使用されます (`libstdc++` または `libc++`). @@ -749,7 +749,7 @@ CPU命令セットは、サーバー間でサポートされる最小のセッ ## 図書館 {#libraries} -**1.** C++14標準ライブラリが使用されています(実験的な拡張が許可されています)。 `boost` と `Poco` フレームワーク +**1.** C++20標準ライブラリが使用されています(実験的な拡張が許可されています)。 `boost` と `Poco` フレームワーク **2.** 必要に応じて、OSパッケージで利用可能な既知のライブラリを使用できます。 diff --git a/docs/ru/development/style.md b/docs/ru/development/style.md index 2cc91d93681..d97b3469467 100644 --- a/docs/ru/development/style.md +++ b/docs/ru/development/style.md @@ -771,9 +771,9 @@ The dictionary is configured incorrectly. Хотя, при прочих равных условиях, предпочитается более-менее кроссплатформенный или легко портируемый код. -**2.** Язык - C++20. +**2.** Язык - C++20 (см. список доступных [C++20 фич](https://en.cppreference.com/w/cpp/compiler_support#C.2B.2B20_features)). -**3.** Компилятор - `gcc`. На данный момент (апрель 2020), код собирается версией 9. (Также код может быть собран `clang` версий 10 и 9) +**3.** Компилятор - `gcc`. На данный момент (август 2020), код собирается версией 9.3. (Также код может быть собран `clang` версий 10 и 9) Используется стандартная библиотека (реализация `libc++`). @@ -815,7 +815,7 @@ The dictionary is configured incorrectly. ## Библиотеки {#biblioteki} -**1.** Используются стандартная библиотека C++14 (допустимо использовать экспериментальные расширения) а также фреймворки `boost`, `Poco`. +**1.** Используются стандартная библиотека C++20 (допустимо использовать экспериментальные расширения) а также фреймворки `boost`, `Poco`. **2.** При необходимости, можно использовать любые известные библиотеки, доступные в ОС из пакетов. diff --git a/docs/tr/development/style.md b/docs/tr/development/style.md index 3e8f5960d64..bf2dd250ad0 100644 --- a/docs/tr/development/style.md +++ b/docs/tr/development/style.md @@ -701,9 +701,9 @@ auto s = std::string{"Hello"}; Ama diğer şeyler eşit olmak, çapraz platform veya taşınabilir kod tercih edilir. -**2.** Dil: C++17. +**2.** Dil: C++20. -**3.** Derleyici: `gcc`. Şu anda (Aralık 2017), kod sürüm 7.2 kullanılarak derlenmiştir. (Ayrıca kullanılarak derlenebilir `clang 4`.) +**3.** Derleyici: `gcc`. Şu anda (Ağustos 2020), kod sürüm 9.3 kullanılarak derlenmiştir. (Ayrıca kullanılarak derlenebilir `clang 8`.) Standart kütüphane kullanılır (`libstdc++` veya `libc++`). @@ -749,7 +749,7 @@ Eğer kod inunuz `master` şube henüz imara değil, önce inşa onu hariç `pus ## Kitaplık {#libraries} -**1.** C++14 standart Kütüphanesi kullanılır (deneysel uzantılara izin verilir) ve `boost` ve `Poco` çerçeveler. +**1.** C++20 standart Kütüphanesi kullanılır (deneysel uzantılara izin verilir) ve `boost` ve `Poco` çerçeveler. **2.** Gerekirse, OS paketinde bulunan iyi bilinen kütüphaneleri kullanabilirsiniz. diff --git a/docs/zh/development/style.md b/docs/zh/development/style.md index 4bb8079c237..8451e5208e3 100644 --- a/docs/zh/development/style.md +++ b/docs/zh/development/style.md @@ -694,9 +694,9 @@ auto s = std::string{"Hello"}; 但在其他条件相同的情况下,首选跨平台或可移植代码。 -**2.** 语言: C++17. +**2.** 语言: C++20. -**3.** 编译器: `gcc`。 此时(2017年12月),代码使用7.2版编译。(它也可以使用`clang 4` 编译) +**3.** 编译器: `gcc`。 此时(2020年08月),代码使用9.3版编译。(它也可以使用`clang 8` 编译) 使用标准库 (`libstdc++` 或 `libc++`)。 @@ -742,7 +742,7 @@ CPU指令集是我们服务器中支持的最小集合。 目前,它是SSE 4.2 ## 库 {#ku} -**1.** 使用C ++ 14标准库(允许实验性功能),以及 `boost` 和 `Poco` 框架。 +**1.** 使用C++20标准库(允许实验性功能),以及 `boost` 和 `Poco` 框架。 **2.** 如有必要,您可以使用 OS 包中提供的任何已知库。 From 81c681fe833772e592ecf6cf3f0b2c64fb980377 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 7 Aug 2020 17:51:20 +0300 Subject: [PATCH 155/374] done --- utils/simple-backport/changelog.sh | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/utils/simple-backport/changelog.sh b/utils/simple-backport/changelog.sh index 33908414235..10fa8ebb8fe 100755 --- a/utils/simple-backport/changelog.sh +++ b/utils/simple-backport/changelog.sh @@ -1,6 +1,7 @@ #!/bin/bash set -e + script_dir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" from="$1" @@ -66,7 +67,11 @@ do # Filter out PRs by bots. user_login=$(jq -r .user.login "$file") - if echo "$user_login" | grep "\[bot\]$" > /dev/null + + filter_bot=$(echo "$user_login" | grep -q "\[bot\]$" && echo "Skip." || echo "Ok." ||:) + filter_robot=$(echo "$user_login" | grep -q "robot-clickhouse" && echo "Skip." || echo "Ok." ||:) + + if [ "Skip." == "$filter_robot" ] || [ "Skip." == "$filter_bot" ] then continue fi From 6a1b0106ca7a7f5b3f7b104cb23887d5232ebf07 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 7 Aug 2020 17:52:50 +0300 Subject: [PATCH 156/374] better --- utils/simple-backport/changelog.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/utils/simple-backport/changelog.sh b/utils/simple-backport/changelog.sh index 10fa8ebb8fe..92957987cab 100755 --- a/utils/simple-backport/changelog.sh +++ b/utils/simple-backport/changelog.sh @@ -1,7 +1,6 @@ #!/bin/bash set -e - script_dir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" from="$1" From c06210a89ac514f228962e5e3df745a3eda96322 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 7 Aug 2020 18:18:32 +0300 Subject: [PATCH 157/374] Avoid overallocation in PODArray assignment Part of #12438 --- src/Common/PODArray.h | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index 1fddea4507b..928ec498d68 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -220,6 +220,13 @@ public: realloc(roundUpToPowerOfTwoOrZero(minimum_memory_for_elements(n)), std::forward(allocator_params)...); } + template + void reserve_exact(size_t n, TAllocatorParams &&... allocator_params) + { + if (n > capacity()) + realloc(minimum_memory_for_elements(n), std::forward(allocator_params)...); + } + template void resize(size_t n, TAllocatorParams &&... allocator_params) { @@ -227,6 +234,13 @@ public: resize_assume_reserved(n); } + template + void resize_exact(size_t n, TAllocatorParams &&... allocator_params) + { + reserve_exact(n, std::forward(allocator_params)...); + resize_assume_reserved(n); + } + void resize_assume_reserved(const size_t n) { c_end = c_start + byte_size(n); @@ -601,7 +615,7 @@ public: template void assign(size_t n, const T & x, TAllocatorParams &&... allocator_params) { - this->resize(n, std::forward(allocator_params)...); + this->resize_exact(n, std::forward(allocator_params)...); std::fill(begin(), end(), x); } @@ -612,7 +626,7 @@ public: size_t required_capacity = from_end - from_begin; if (required_capacity > this->capacity()) - this->reserve(roundUpToPowerOfTwoOrZero(required_capacity), std::forward(allocator_params)...); + this->reserve_exact(required_capacity, std::forward(allocator_params)...); size_t bytes_to_copy = this->byte_size(required_capacity); memcpy(this->c_start, reinterpret_cast(&*from_begin), bytes_to_copy); From 562102161b651f23701181ec36741b05bcf444c5 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 7 Aug 2020 18:53:40 +0300 Subject: [PATCH 158/374] Missed override. --- src/Disks/S3/DiskS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 66db32a4837..26f4dad4303 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -421,7 +421,7 @@ class AsyncExecutor : public Executor public: explicit AsyncExecutor() = default; - std::future execute(std::function task) + std::future execute(std::function task) override { auto promise = std::make_shared>(); From 6da8633741b20536184013b7a8c699f832cd94f1 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 8 Aug 2020 00:14:10 +0800 Subject: [PATCH 159/374] ISSUES-4006 try fix build failure & test failure --- src/Storages/StorageMaterializeMySQL.cpp | 8 ++++++++ src/Storages/StorageMaterializeMySQL.h | 6 ++++++ 2 files changed, 14 insertions(+) diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index ae7745e945b..a855fccef56 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#if USE_MYSQL + #include #include @@ -114,3 +120,5 @@ NamesAndTypesList StorageMaterializeMySQL::getVirtuals() const } } + +#endif diff --git a/src/Storages/StorageMaterializeMySQL.h b/src/Storages/StorageMaterializeMySQL.h index 62909e01a36..39eca3191f1 100644 --- a/src/Storages/StorageMaterializeMySQL.h +++ b/src/Storages/StorageMaterializeMySQL.h @@ -1,5 +1,9 @@ #pragma once +#include "config_core.h" + +#if USE_MYSQL + #include #include @@ -30,3 +34,5 @@ private: }; } + +#endif From 7d1283247f2f2add53021ddf2f8de5e438e8d7bd Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 7 Aug 2020 20:16:59 +0300 Subject: [PATCH 160/374] Fix tests --- .../test_adaptive_granularity/test.py | 17 +++++++++++------ .../test.py | 6 +++--- .../test.py | 4 ++-- .../integration/test_polymorphic_parts/test.py | 13 ++++++------- ...ace_partition_with_different_granularity.sql | 14 -------------- 5 files changed, 22 insertions(+), 32 deletions(-) diff --git a/tests/integration/test_adaptive_granularity/test.py b/tests/integration/test_adaptive_granularity/test.py index cd7d6045623..671cb5a672b 100644 --- a/tests/integration/test_adaptive_granularity/test.py +++ b/tests/integration/test_adaptive_granularity/test.py @@ -301,11 +301,16 @@ def test_mixed_granularity_single_node(start_dynamic_cluster, node): def test_version_update_two_nodes(start_dynamic_cluster): node11.query("INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)") - node12.query("SYSTEM SYNC REPLICA table_with_default_granularity") + node12.query("SYSTEM SYNC REPLICA table_with_default_granularity", timeout=10) assert node12.query("SELECT COUNT() FROM table_with_default_granularity") == '2\n' - node12.restart_with_latest_version() + def callback(n): + n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", "0") + n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml", "0") + + node12.restart_with_latest_version(callback_onstop=callback) + node12.query("INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 3, 333), (toDate('2018-10-02'), 4, 444)") - node11.query("SYSTEM SYNC REPLICA table_with_default_granularity") + node11.query("SYSTEM SYNC REPLICA table_with_default_granularity", timeout=10) assert node11.query("SELECT COUNT() FROM table_with_default_granularity") == '4\n' node12.query( @@ -329,14 +334,14 @@ def test_version_update_two_nodes(start_dynamic_cluster): node11.query("SYSTEM SYNC REPLICA table_with_default_granularity_new", timeout=5) node12.query("INSERT INTO table_with_default_granularity_new VALUES (toDate('2018-10-01'), 3, 333), (toDate('2018-10-02'), 4, 444)") - node11.restart_with_latest_version() # just to be sure + node11.restart_with_latest_version(callback_onstop=callback) # just to be sure node11.query("SYSTEM SYNC REPLICA table_with_default_granularity_new", timeout=5) node12.query("SYSTEM SYNC REPLICA table_with_default_granularity_new", timeout=5) node11.query("SELECT COUNT() FROM table_with_default_granularity_new") == "4\n" node12.query("SELECT COUNT() FROM table_with_default_granularity_new") == "4\n" - node11.query("SYSTEM SYNC REPLICA table_with_default_granularity") + node11.query("SYSTEM SYNC REPLICA table_with_default_granularity", timeout=5) node11.query("INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 5, 333), (toDate('2018-10-02'), 6, 444)") - node12.query("SYSTEM SYNC REPLICA table_with_default_granularity") + node12.query("SYSTEM SYNC REPLICA table_with_default_granularity", timeout=5) assert node12.query("SELECT COUNT() FROM table_with_default_granularity") == '6\n' diff --git a/tests/integration/test_adaptive_granularity_different_settings/test.py b/tests/integration/test_adaptive_granularity_different_settings/test.py index d1ff785cfba..23c95ef1701 100644 --- a/tests/integration/test_adaptive_granularity_different_settings/test.py +++ b/tests/integration/test_adaptive_granularity_different_settings/test.py @@ -32,12 +32,12 @@ def test_attach_detach(start_cluster): node2.query(""" CREATE TABLE test (key UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test', '2') - ORDER BY tuple()""") + ORDER BY tuple() SETTINGS enable_mixed_granularity_parts = 0""") node2.query("INSERT INTO test VALUES (3), (4)") - node1.query("SYSTEM SYNC REPLICA test") - node2.query("SYSTEM SYNC REPLICA test") + node1.query("SYSTEM SYNC REPLICA test", timeout=10) + node2.query("SYSTEM SYNC REPLICA test", timeout=10) assert node1.query("SELECT COUNT() FROM test") == "4\n" assert node2.query("SELECT COUNT() FROM test") == "4\n" diff --git a/tests/integration/test_adaptive_granularity_replicated/test.py b/tests/integration/test_adaptive_granularity_replicated/test.py index a0bfa2e0b76..52978ade685 100644 --- a/tests/integration/test_adaptive_granularity_replicated/test.py +++ b/tests/integration/test_adaptive_granularity_replicated/test.py @@ -27,7 +27,7 @@ def test_creating_table_different_setting(start_cluster): node1.query("CREATE TABLE t1 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t1', '1') ORDER BY tuple(c1) SETTINGS index_granularity_bytes = 0") node1.query("INSERT INTO t1 VALUES('x', 'y')") - node2.query("CREATE TABLE t1 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t1', '2') ORDER BY tuple(c1)") + node2.query("CREATE TABLE t1 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t1', '2') ORDER BY tuple(c1) SETTINGS enable_mixed_granularity_parts = 0") node1.query("INSERT INTO t1 VALUES('a', 'b')") node2.query("SYSTEM SYNC REPLICA t1", timeout=5) @@ -64,7 +64,7 @@ def test_old_node_with_new_node(start_cluster): node3.query("CREATE TABLE t2 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t2', '3') ORDER BY tuple(c1)") node3.query("INSERT INTO t2 VALUES('x', 'y')") - node2.query("CREATE TABLE t2 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t2', '2') ORDER BY tuple(c1)") + node2.query("CREATE TABLE t2 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t2', '2') ORDER BY tuple(c1) SETTINGS enable_mixed_granularity_parts = 0") node3.query("INSERT INTO t2 VALUES('a', 'b')") node2.query("SYSTEM SYNC REPLICA t2", timeout=5) diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index 072223e6380..d3ebbd8c7a8 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -29,7 +29,7 @@ def insert_random_data(table, node, size): str(get_random_array()))) + ')' for i in range(size) ] - + node.query("INSERT INTO {} VALUES {}".format(table, ','.join(data))) def create_tables(name, nodes, node_settings, shard): @@ -40,7 +40,7 @@ def create_tables(name, nodes, node_settings, shard): ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/{name}', '{repl}') PARTITION BY toYYYYMM(date) ORDER BY id - SETTINGS index_granularity = 64, index_granularity_bytes = {index_granularity_bytes}, + SETTINGS index_granularity = 64, index_granularity_bytes = {index_granularity_bytes}, min_rows_for_wide_part = {min_rows_for_wide_part}, min_rows_for_compact_part = {min_rows_for_compact_part}, in_memory_parts_enable_wal = 1 '''.format(name=name, shard=shard, repl=i, **settings)) @@ -84,7 +84,7 @@ def start_cluster(): create_tables('polymorphic_table', [node1, node2], [settings_default, settings_default], "shard1") create_tables('compact_parts_only', [node1, node2], [settings_compact_only, settings_compact_only], "shard1") - create_tables('non_adaptive_table', [node1, node2], [settings_not_adaptive, settings_default], "shard1") + create_tables('non_adaptive_table', [node1, node2], [settings_not_adaptive, settings_not_adaptive], "shard1") create_tables('polymorphic_table_compact', [node3, node4], [settings_compact, settings_wide], "shard2") create_tables('polymorphic_table_wide', [node3, node4], [settings_wide, settings_compact], "shard2") create_tables_old_format('polymorphic_table', [node5, node6], "shard3") @@ -184,7 +184,6 @@ def test_compact_parts_only(start_cluster): assert TSV(node2.query("SELECT part_type, count() FROM system.parts " \ "WHERE table = 'compact_parts_only' AND active GROUP BY part_type ORDER BY part_type")) == TSV(expected) - # Check that follower replicas create parts of the same type, which leader has chosen at merge. @pytest.mark.parametrize( ('table', 'part_type'), @@ -268,7 +267,7 @@ def test_polymorphic_parts_diff_versions(start_cluster_diff_versions): @pytest.mark.skip(reason="compatability is temporary broken") def test_polymorphic_parts_diff_versions_2(start_cluster_diff_versions): - # Replication doesn't work on old version if part is created in compact format, because + # Replication doesn't work on old version if part is created in compact format, because # this version doesn't know anything about it. It's considered to be ok. node_old = node7 @@ -465,8 +464,8 @@ def test_in_memory_alters(start_cluster): def test_polymorphic_parts_index(start_cluster): node1.query(''' - CREATE TABLE index_compact(a UInt32, s String) - ENGINE = MergeTree ORDER BY a + CREATE TABLE index_compact(a UInt32, s String) + ENGINE = MergeTree ORDER BY a SETTINGS min_rows_for_wide_part = 1000, index_granularity = 128, merge_max_block_size = 100''') node1.query("INSERT INTO index_compact SELECT number, toString(number) FROM numbers(100)") diff --git a/tests/queries/1_stateful/00151_replace_partition_with_different_granularity.sql b/tests/queries/1_stateful/00151_replace_partition_with_different_granularity.sql index bea90dade3c..fa18b53bdf9 100644 --- a/tests/queries/1_stateful/00151_replace_partition_with_different_granularity.sql +++ b/tests/queries/1_stateful/00151_replace_partition_with_different_granularity.sql @@ -16,20 +16,6 @@ OPTIMIZE TABLE mixed_granularity_table FINAL; SELECT COUNT() FROM mixed_granularity_table; --- check strange cases when we try to replace parts from another tables but with different granularity settings - -DROP TABLE IF EXISTS non_mixed_granularity_adaptive_table; - -CREATE TABLE non_mixed_granularity_adaptive_table AS test.hits; - -INSERT INTO non_mixed_granularity_adaptive_table SELECT * FROM test.hits LIMIT 10; - -ALTER TABLE non_mixed_granularity_adaptive_table REPLACE PARTITION 201403 FROM test.hits; -- { serverError 36 } - -DROP TABLE IF EXISTS non_mixed_granularity_adaptive_table; - -DROP TABLE IF EXISTS non_mixed_granularity_non_adaptive_table; - CREATE TABLE non_mixed_granularity_non_adaptive_table (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0; -- same with hits, but enabled mixed granularity and fixed_granularity INSERT INTO non_mixed_granularity_non_adaptive_table SELECT * FROM test.hits LIMIT 10; From e3235e5b6ed59a5e7361383c1845e46189c31859 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Aug 2020 22:06:31 +0300 Subject: [PATCH 161/374] Fix build --- src/AggregateFunctions/IAggregateFunction.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 899c7d3e8b8..cb5f6604a93 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -360,7 +360,7 @@ public: static constexpr size_t UNROLL_COUNT = 4; - Data places[256 * UNROLL_COUNT]; + std::unique_ptr places{new Data[256 * UNROLL_COUNT]}; bool has_data[256 * UNROLL_COUNT]{}; /// Separate flags array to avoid heavy initialization. size_t i = 0; From b6625006dcf0ad016d7502aa0a958569cd4ee93c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Aug 2020 22:36:04 +0300 Subject: [PATCH 162/374] Hardening and better error messages --- src/Columns/ColumnArray.cpp | 7 ++++--- src/Columns/ColumnString.cpp | 16 ++++++++++++++++ src/Columns/ColumnString.h | 5 +---- src/Common/Allocator.h | 12 ++++++++++++ src/Common/MemoryTracker.cpp | 4 ++++ 5 files changed, 37 insertions(+), 7 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 1deb89b1492..7e4e6233f73 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -42,11 +42,12 @@ ColumnArray::ColumnArray(MutableColumnPtr && nested_column, MutableColumnPtr && if (!offsets_concrete) throw Exception("offsets_column must be a ColumnUInt64", ErrorCodes::LOGICAL_ERROR); - size_t size = offsets_concrete->size(); - if (size != 0 && nested_column) + if (!offsets_concrete->empty() && nested_column) { + Offset last_offset = offsets_concrete->getData().back(); + /// This will also prevent possible overflow in offset. - if (nested_column->size() != offsets_concrete->getData()[size - 1]) + if (nested_column->size() != last_offset) throw Exception("offsets_column has data inconsistent with nested_column", ErrorCodes::LOGICAL_ERROR); } diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 467a8681868..5e0890e11e6 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -22,6 +22,22 @@ namespace ErrorCodes } +ColumnString::ColumnString(const ColumnString & src) + : offsets(src.offsets.begin(), src.offsets.end()), + chars(src.chars.begin(), src.chars.end()) +{ + if (!offsets.empty()) + { + Offset last_offset = offsets.back(); + + /// This will also prevent possible overflow in offset. + if (chars.size() != last_offset) + throw Exception("String offsets has data inconsistent with chars array", ErrorCodes::LOGICAL_ERROR); + } +} + + + MutableColumnPtr ColumnString::cloneResized(size_t to_size) const { auto res = ColumnString::create(); diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index d8948afacbc..128e1efe146 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -49,10 +49,7 @@ private: struct lessWithCollation; ColumnString() = default; - - ColumnString(const ColumnString & src) - : offsets(src.offsets.begin(), src.offsets.end()), - chars(src.chars.begin(), src.chars.end()) {} + ColumnString(const ColumnString & src); public: const char * getFamilyName() const override { return "String"; } diff --git a/src/Common/Allocator.h b/src/Common/Allocator.h index a593783a047..6ac60020379 100644 --- a/src/Common/Allocator.h +++ b/src/Common/Allocator.h @@ -70,6 +70,7 @@ namespace ErrorCodes extern const int CANNOT_ALLOCATE_MEMORY; extern const int CANNOT_MUNMAP; extern const int CANNOT_MREMAP; + extern const int LOGICAL_ERROR; } } @@ -90,6 +91,7 @@ public: /// Allocate memory range. void * alloc(size_t size, size_t alignment = 0) { + checkSize(size); CurrentMemoryTracker::alloc(size); return allocNoTrack(size, alignment); } @@ -97,6 +99,7 @@ public: /// Free memory range. void free(void * buf, size_t size) { + checkSize(size); freeNoTrack(buf, size); CurrentMemoryTracker::free(size); } @@ -107,6 +110,8 @@ public: */ void * realloc(void * buf, size_t old_size, size_t new_size, size_t alignment = 0) { + checkSize(new_size); + if (old_size == new_size) { /// nothing to do. @@ -244,6 +249,13 @@ private: } } + void checkSize(size_t size) + { + /// More obvious exception in case of possible overflow (instead of just "Cannot mmap"). + if (size >= 0x8000000000000000ULL) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Too large size ({}) passed to allocator. It indicates an error.", size); + } + #ifndef NDEBUG /// In debug builds, request mmap() at random addresses (a kind of ASLR), to /// reproduce more memory stomping bugs. Note that Linux doesn't do it by diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index ca21cec0212..e8c009658eb 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -18,6 +18,7 @@ namespace DB namespace ErrorCodes { extern const int MEMORY_LIMIT_EXCEEDED; + extern const int LOGICAL_ERROR; } } @@ -62,6 +63,9 @@ void MemoryTracker::logMemoryUsage(Int64 current) const void MemoryTracker::alloc(Int64 size) { + if (size < 0) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Negative size ({}) is passed to MemoryTracker. It is a bug.", size); + if (blocker.isCancelled()) return; From 9938e1f3f680f5c346ca38cdfc06ccadc9b5b510 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 8 Aug 2020 01:38:42 +0800 Subject: [PATCH 163/374] Better toDate/toDateTime function. --- src/Functions/FunctionsConversion.h | 122 ++++++++++++++++-- .../01440_to_date_monotonicity.reference | 2 + .../01440_to_date_monotonicity.sql | 9 ++ 3 files changed, 124 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/01440_to_date_monotonicity.reference create mode 100644 tests/queries/0_stateless/01440_to_date_monotonicity.sql diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index c2a7f3f3cd2..5c3960fdef5 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -196,12 +196,38 @@ struct ToDateTransform32Or64 static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { - return (from < 0xFFFF) ? from : time_zone.toDayNum(from); + return (from < 0xFFFF) ? from : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); } }; -/** Special case of converting (U)Int32 or (U)Int64 (and also, for convenience, Float32, Float64) to Date. - * If number is less than 65536, then it is treated as DayNum, and if greater or equals, then as unix timestamp. +template +struct ToDateTransform32Or64Signed +{ + static constexpr auto name = "toDate"; + + static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) + { + if (from < 0) return 0; + return (from < 0xFFFF) ? from : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); + } +}; + +template +struct ToDateTransform8Or16Signed +{ + static constexpr auto name = "toDate"; + + static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) + { + if (from < 0) return 0; + return from; + } +}; + +/** Special case of converting Int8, Int16, (U)Int32 or (U)Int64 (and also, for convenience, + * Float32, Float64) to Date. If the number is negative, saturate it to unix epoch time. If the + * number is less than 65536, then it is treated as DayNum, and if it's greater or equals to 65536, + * then treated as unix timestamp. If the number exceeds UInt32, saturate to MAX_UINT32 then as DayNum. * It's a bit illogical, as we actually have two functions in one. * But allows to support frequent case, * when user write toDate(UInt32), expecting conversion of unix timestamp to Date. @@ -211,14 +237,73 @@ template struct ConvertImpl : DateTimeTransformImpl> {}; template struct ConvertImpl : DateTimeTransformImpl> {}; +template struct ConvertImpl + : DateTimeTransformImpl> {}; +template struct ConvertImpl + : DateTimeTransformImpl> {}; template struct ConvertImpl - : DateTimeTransformImpl> {}; + : DateTimeTransformImpl> {}; template struct ConvertImpl - : DateTimeTransformImpl> {}; + : DateTimeTransformImpl> {}; template struct ConvertImpl - : DateTimeTransformImpl> {}; + : DateTimeTransformImpl> {}; template struct ConvertImpl - : DateTimeTransformImpl> {}; + : DateTimeTransformImpl> {}; + + +template +struct ToDateTimeTransform64 +{ + static constexpr auto name = "toDateTime"; + + static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) + { + return std::min(time_t(from), time_t(0xFFFFFFFF)); + } +}; + +template +struct ToDateTimeTransformSigned +{ + static constexpr auto name = "toDateTime"; + + static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) + { + if (from < 0) return 0; + return from; + } +}; + +template +struct ToDateTimeTransform64Signed +{ + static constexpr auto name = "toDateTime"; + + static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) + { + if (from < 0) return 0; + return std::min(time_t(from), time_t(0xFFFFFFFF)); + } +}; + +/** Special case of converting Int8, Int16, Int32 or (U)Int64 (and also, for convenience, Float32, + * Float64) to DateTime. If the number is negative, saturate it to unix epoch time. If the number + * exceeds UInt32, saturate to MAX_UINT32. + */ +template struct ConvertImpl + : DateTimeTransformImpl> {}; +template struct ConvertImpl + : DateTimeTransformImpl> {}; +template struct ConvertImpl + : DateTimeTransformImpl> {}; +template struct ConvertImpl + : DateTimeTransformImpl> {}; +template struct ConvertImpl + : DateTimeTransformImpl> {}; +template struct ConvertImpl + : DateTimeTransformImpl> {}; +template struct ConvertImpl + : DateTimeTransformImpl> {}; /** Conversion of Date or DateTime to DateTime64: add zero sub-second part. @@ -1412,6 +1497,25 @@ struct ToNumberMonotonicity } }; +struct ToDateMonotonicity +{ + static bool has() { return true; } + + static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right) + { + auto which = WhichDataType(type); + if (which.isDateOrDateTime() || which.isInt8() || which.isInt16() || which.isUInt8() || which.isUInt16()) + return {true, true, true}; + else if ( + (which.isUInt() && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF))) + || (which.isInt() && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF))) + || (which.isFloat() && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF)))) + return {}; + else + return {true, true, true}; + } +}; + /** The monotonicity for the `toString` function is mainly determined for test purposes. * It is doubtful that anyone is looking to optimize queries with conditions `toString(CounterID) = 34`. */ @@ -1478,8 +1582,8 @@ using FunctionToInt32 = FunctionConvert>; using FunctionToFloat32 = FunctionConvert>; using FunctionToFloat64 = FunctionConvert>; -using FunctionToDate = FunctionConvert>; -using FunctionToDateTime = FunctionConvert>; +using FunctionToDate = FunctionConvert; +using FunctionToDateTime = FunctionConvert; using FunctionToDateTime64 = FunctionConvert; using FunctionToUUID = FunctionConvert>; using FunctionToString = FunctionConvert; diff --git a/tests/queries/0_stateless/01440_to_date_monotonicity.reference b/tests/queries/0_stateless/01440_to_date_monotonicity.reference new file mode 100644 index 00000000000..529601fb398 --- /dev/null +++ b/tests/queries/0_stateless/01440_to_date_monotonicity.reference @@ -0,0 +1,2 @@ +0 +1970-01-01 2106-02-07 1970-04-11 1970-01-01 2106-02-07 diff --git a/tests/queries/0_stateless/01440_to_date_monotonicity.sql b/tests/queries/0_stateless/01440_to_date_monotonicity.sql new file mode 100644 index 00000000000..0355d1fec30 --- /dev/null +++ b/tests/queries/0_stateless/01440_to_date_monotonicity.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS tdm; +CREATE TABLE tdm (x DateTime) ENGINE = MergeTree ORDER BY x SETTINGS write_final_mark = 0; +INSERT INTO tdm VALUES (now()); +SELECT count(x) FROM tdm WHERE toDate(x) < today() SETTINGS max_rows_to_read = 1; + +SELECT toDate(-1), toDate(10000000000000), toDate(100), toDate(65536), toDate(65535); +SELECT toDateTime(-1), toDateTime(10000000000000), toDateTime(1000); + +DROP TABLE tdm; From f968ae8804bbd773cf199ff1062a72921c5f448f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Aug 2020 22:45:29 +0300 Subject: [PATCH 164/374] Hardening continued --- src/Functions/repeat.cpp | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index e411f3fdf52..18350dd6207 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -28,6 +28,14 @@ struct RepeatImpl ErrorCodes::TOO_LARGE_STRING_SIZE); } + static inline void checkStringSize(UInt64 size) + { + static constexpr UInt64 max_string_size = 1 << 30; + if (size > max_string_size) + throw Exception("Too large string size (" + std::to_string(size) + ") in function repeat, maximum is: " + std::to_string(max_string_size), + ErrorCodes::TOO_LARGE_STRING_SIZE); + } + static void vectorStrConstRepeat( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, @@ -41,7 +49,10 @@ struct RepeatImpl res_offsets.assign(offsets); for (UInt64 i = 0; i < offsets.size(); ++i) { - data_size += (offsets[i] - offsets[i - 1] - 1) * repeat_time + 1; /// Note that accessing -1th element is valid for PaddedPODArray. + /// Note that accessing -1th element is valid for PaddedPODArray. + size_t repeated_size = (offsets[i] - offsets[i - 1] - 1) * repeat_time + 1; + checkStringSize(repeated_size); + data_size += repeated_size; res_offsets[i] = data_size; } res_data.resize(data_size); @@ -63,7 +74,9 @@ struct RepeatImpl res_offsets.assign(offsets); for (UInt64 i = 0; i < col_num.size(); ++i) { - data_size += (offsets[i] - offsets[i - 1] - 1) * col_num[i] + 1; + size_t repeated_size = (offsets[i] - offsets[i - 1] - 1) * col_num[i] + 1; + checkStringSize(repeated_size); + data_size += repeated_size; res_offsets[i] = data_size; } res_data.resize(data_size); @@ -89,7 +102,9 @@ struct RepeatImpl UInt64 col_size = col_num.size(); for (UInt64 i = 0; i < col_size; ++i) { - data_size += str_size * col_num[i] + 1; + size_t repeated_size = str_size * col_num[i] + 1; + checkStringSize(repeated_size); + data_size += repeated_size; res_offsets[i] = data_size; } res_data.resize(data_size); From 65b3863a9e525e4eb57d951d3be6f365e40df5c4 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 7 Aug 2020 22:52:21 +0300 Subject: [PATCH 165/374] Update FunctionsConversion.h --- src/Functions/FunctionsConversion.h | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 5c3960fdef5..b67d023726d 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -196,7 +196,9 @@ struct ToDateTransform32Or64 static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { - return (from < 0xFFFF) ? from : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); + return (from < 0xFFFF) + ? from + : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); } }; @@ -207,8 +209,12 @@ struct ToDateTransform32Or64Signed static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { - if (from < 0) return 0; - return (from < 0xFFFF) ? from : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); + /// The function should be monotonic (better for query optimizations), so we saturate instead of overflow. + if (from < 0) + return 0; + return (from < 0xFFFF) + ? from + : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); } }; @@ -219,7 +225,8 @@ struct ToDateTransform8Or16Signed static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { - if (from < 0) return 0; + if (from < 0) + return 0; return from; } }; From cbe9878e9e322fd2e50839904cefe3cac8612072 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 7 Aug 2020 22:53:52 +0300 Subject: [PATCH 166/374] Update FunctionsConversion.h --- src/Functions/FunctionsConversion.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index b67d023726d..7b1b8388866 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -276,7 +276,8 @@ struct ToDateTimeTransformSigned static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { - if (from < 0) return 0; + if (from < 0) + return 0; return from; } }; @@ -288,7 +289,8 @@ struct ToDateTimeTransform64Signed static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { - if (from < 0) return 0; + if (from < 0) + return 0; return std::min(time_t(from), time_t(0xFFFFFFFF)); } }; From d026d4e28996298f17fd36547fdb5482aa3e1666 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Aug 2020 23:53:46 +0300 Subject: [PATCH 167/374] Fix error --- src/Functions/URL/netloc.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Functions/URL/netloc.cpp b/src/Functions/URL/netloc.cpp index ea45504fa18..b8910d1a20c 100644 --- a/src/Functions/URL/netloc.cpp +++ b/src/Functions/URL/netloc.cpp @@ -53,10 +53,11 @@ struct ExtractNetloc case '&': return StringRef{}; default: - pos = scheme_end; /// exit from the loop + goto exloop; } } } + exloop: if (pos + 2 < scheme_end && pos[0] == ':' && pos[1] == '/' && pos[2] == '/') pos += 3; else From df79f415ce24e650244150bc399ac083def3d05d Mon Sep 17 00:00:00 2001 From: Konstantin Podshumok Date: Sat, 8 Aug 2020 00:28:10 +0300 Subject: [PATCH 168/374] Allow -march=native with clang --- CMakeLists.txt | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index e137c2267f0..9d1f46833b5 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -176,10 +176,7 @@ if(NOT DISABLE_CPU_OPTIMIZE) include(cmake/cpu_features.cmake) endif() -if(NOT COMPILER_CLANG) # clang: error: the clang compiler does not support '-march=native' - option(ARCH_NATIVE "Enable -march=native compiler flag" ${ARCH_ARM}) -endif() - +option(ARCH_NATIVE "Enable -march=native compiler flag" ${ARCH_ARM}) if (ARCH_NATIVE) set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native") endif () From ec281cd7032794e63249f63121277bde6b9d304c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 02:01:05 +0300 Subject: [PATCH 169/374] Fix assert in "lcm" --- src/Functions/lcm.cpp | 40 +++++++++++++++++-- .../0_stateless/01435_lcm_overflow.reference | 8 ++++ .../0_stateless/01435_lcm_overflow.sql | 10 +++++ 3 files changed, 55 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01435_lcm_overflow.reference create mode 100644 tests/queries/0_stateless/01435_lcm_overflow.sql diff --git a/src/Functions/lcm.cpp b/src/Functions/lcm.cpp index 9230a292c43..98399b8374b 100644 --- a/src/Functions/lcm.cpp +++ b/src/Functions/lcm.cpp @@ -1,6 +1,28 @@ #include #include + #include +#include +#include + + +namespace +{ + +template +constexpr T abs(T value) noexcept +{ + if constexpr (std::is_signed_v) + { + if (value >= 0 || value == std::numeric_limits::min()) + return value; + return -value; + } + else + return value; +} + +} namespace DB @@ -18,9 +40,21 @@ struct LCMImpl { throwIfDivisionLeadsToFPE(typename NumberTraits::ToInteger::Type(a), typename NumberTraits::ToInteger::Type(b)); throwIfDivisionLeadsToFPE(typename NumberTraits::ToInteger::Type(b), typename NumberTraits::ToInteger::Type(a)); - return std::lcm( - typename NumberTraits::ToInteger::Type(a), - typename NumberTraits::ToInteger::Type(b)); + + /** It's tempting to use std::lcm function. + * But it has undefined behaviour on overflow. + * And assert in debug build. + * We need some well defined behaviour instead + * (example: throw an exception or overflow in implementation specific way). + */ + + using Int = typename NumberTraits::ToInteger::Type; + + Int val1 = abs(a) / std::gcd(Int(a), Int(b)); + Int val2 = abs(b); + + /// Overflow in implementation specific way. + return Result(val1 * val2); } #if USE_EMBEDDED_COMPILER diff --git a/tests/queries/0_stateless/01435_lcm_overflow.reference b/tests/queries/0_stateless/01435_lcm_overflow.reference new file mode 100644 index 00000000000..56401b21045 --- /dev/null +++ b/tests/queries/0_stateless/01435_lcm_overflow.reference @@ -0,0 +1,8 @@ +30 +30 +30 +30 +18446744073709551360 +-256 +-256 +-256 diff --git a/tests/queries/0_stateless/01435_lcm_overflow.sql b/tests/queries/0_stateless/01435_lcm_overflow.sql new file mode 100644 index 00000000000..3285514ac1d --- /dev/null +++ b/tests/queries/0_stateless/01435_lcm_overflow.sql @@ -0,0 +1,10 @@ +SELECT lcm(15, 10); +SELECT lcm(-15, 10); +SELECT lcm(15, -10); +SELECT lcm(-15, -10); + +-- Implementation specific result on overflow: +SELECT lcm(256, 9223372036854775807); +SELECT lcm(256, -9223372036854775807); +SELECT lcm(-256, 9223372036854775807); +SELECT lcm(-256, -9223372036854775807); From 378b2b7e04b2d40e67904f5127febeb1ac480e60 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 8 Aug 2020 02:06:55 +0300 Subject: [PATCH 170/374] Update average.cpp --- src/Common/tests/average.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/tests/average.cpp b/src/Common/tests/average.cpp index e1a617524b3..af85fb8d9d9 100644 --- a/src/Common/tests/average.cpp +++ b/src/Common/tests/average.cpp @@ -515,7 +515,7 @@ int main(int argc, char ** argv) /// Aggregate Stopwatch watch; - Float res; + Float res{}; switch (variant) { From 94fcd929db8c31aa8a113682acbab9f881fad495 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 03:07:33 +0300 Subject: [PATCH 171/374] Update test --- .../0_stateless/01404_roundUpToPowerOfTwoOrZero_safety.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01404_roundUpToPowerOfTwoOrZero_safety.sql b/tests/queries/0_stateless/01404_roundUpToPowerOfTwoOrZero_safety.sql index d235b045e11..4ee6e1fa5e4 100644 --- a/tests/queries/0_stateless/01404_roundUpToPowerOfTwoOrZero_safety.sql +++ b/tests/queries/0_stateless/01404_roundUpToPowerOfTwoOrZero_safety.sql @@ -1,4 +1,4 @@ -- repeat() with this length and this number of rows will allocation huge enough region (MSB set), -- which will cause roundUpToPowerOfTwoOrZero() returns 0 for such allocation (before the fix), -- and later repeat() will try to use this memory and will got SIGSEGV. -SELECT repeat('0.0001048576', number * (number * (number * 255))) FROM numbers(65535); -- { serverError 173; } +SELECT repeat('0.0001048576', number * (number * (number * 255))) FROM numbers(65535); -- { serverError 131; } From ac2de4cc950cfa9ea3a86834143a162c63ab274b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 03:08:41 +0300 Subject: [PATCH 172/374] Fix build --- src/Functions/URL/netloc.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/URL/netloc.cpp b/src/Functions/URL/netloc.cpp index b8910d1a20c..cb28083f4c7 100644 --- a/src/Functions/URL/netloc.cpp +++ b/src/Functions/URL/netloc.cpp @@ -69,7 +69,7 @@ struct ExtractNetloc bool has_identification = false; Pos question_mark_pos = end; Pos slash_pos = end; - auto start_of_host = pos; + Pos start_of_host = pos; for (; pos < end; ++pos) { switch (*pos) From edd89a861051569c9f1a77e413590e1c8db66c19 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 03:47:03 +0300 Subject: [PATCH 173/374] Fix half of typos --- src/Access/AllowedClientHosts.cpp | 2 +- src/Access/IAccessStorage.h | 4 +-- .../AggregateFunctionGroupBitmapData.h | 12 +++---- .../AggregateFunctionMLMethod.cpp | 2 +- .../AggregateFunctionNull.h | 2 +- .../AggregateFunctionSequenceMatch.h | 2 +- .../AggregateFunctionSumMap.cpp | 4 +-- .../AggregateFunctionWindowFunnel.h | 4 +-- .../IAggregateFunctionCombinator.h | 2 +- src/Client/TimeoutSetter.cpp | 2 +- src/Columns/IColumn.h | 4 +-- src/Common/ArrayCache.h | 2 +- src/Common/AutoArray.h | 2 +- src/Common/Config/ConfigReloader.cpp | 2 +- src/Common/Config/ConfigReloader.h | 2 +- src/Common/CounterInFile.h | 2 +- src/Common/CurrentMetrics.cpp | 4 +-- src/Common/CurrentThread.h | 2 +- src/Common/DNSResolver.cpp | 2 +- src/Common/FieldVisitors.h | 2 +- src/Common/HashTable/FixedHashTable.h | 4 +-- src/Common/LRUCache.h | 2 +- src/Common/MemoryTracker.cpp | 2 +- src/Common/NamePrompter.h | 10 +++--- src/Common/ObjectPool.h | 2 +- src/Common/PODArray.h | 2 +- src/Common/QueryProfiler.h | 2 +- src/Common/RWLock.cpp | 2 +- src/Common/SensitiveDataMasker.h | 2 +- src/Common/StackTrace.cpp | 2 +- src/Common/SymbolIndex.cpp | 6 ++-- src/Common/TaskStatsInfoGetter.h | 2 +- src/Common/Volnitsky.h | 32 ++++++++++--------- src/Common/ZooKeeper/ZooKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 +- src/Common/formatIPv6.h | 4 +-- src/Common/intExp.h | 2 +- src/Common/parseGlobs.cpp | 2 +- src/Common/tests/chaos_sanitizer.cpp | 2 +- src/Compression/CompressedReadBufferBase.cpp | 2 +- .../CompressionCodecDoubleDelta.cpp | 2 +- .../tests/gtest_compressionCodec.cpp | 2 +- src/Core/BlockInfo.h | 2 +- src/Core/DecimalFunctions.h | 2 +- src/Core/Defines.h | 2 +- src/Core/Settings.h | 2 +- .../AddingDefaultsBlockInputStream.cpp | 4 +-- .../AsynchronousBlockInputStream.h | 2 +- .../MergingSortedBlockInputStream.cpp | 2 +- .../MergingSortedBlockInputStream.h | 2 +- src/DataStreams/ParallelInputsProcessor.h | 2 +- .../ParallelParsingBlockInputStream.cpp | 4 +-- .../PushingToViewsBlockOutputStream.cpp | 2 +- .../DataTypeCustomSimpleAggregateFunction.h | 2 +- src/DataTypes/DataTypeDateTime64.h | 2 +- src/DataTypes/IDataType.h | 2 +- src/DataTypes/NestedUtils.cpp | 18 +++++------ src/DataTypes/getLeastSupertype.cpp | 2 +- src/Dictionaries/BucketCache.h | 4 +-- src/Dictionaries/CacheDictionary.h | 8 ++--- src/Dictionaries/ComplexKeyCacheDictionary.h | 2 +- src/Dictionaries/PolygonDictionaryUtils.cpp | 2 +- src/Dictionaries/PolygonDictionaryUtils.h | 2 +- src/Functions/FunctionBinaryArithmetic.h | 2 +- src/Functions/FunctionCustomWeekToSomething.h | 6 ++-- .../FunctionDateOrDateTimeAddInterval.h | 2 +- src/Functions/FunctionsBitmap.h | 6 ++-- src/Functions/FunctionsConversion.h | 10 +++--- src/Functions/FunctionsHashing.h | 4 +-- src/Functions/PerformanceAdaptors.h | 2 +- src/Functions/PolygonUtils.h | 6 ++-- src/Functions/TargetSpecific.h | 4 +-- src/Functions/abtesting.cpp | 6 ++-- src/Functions/array/arrayEnumerateRanked.cpp | 2 +- src/Functions/array/arrayEnumerateRanked.h | 2 +- src/Functions/extractAllGroups.h | 2 +- src/Functions/formatDateTime.cpp | 2 +- src/Functions/formatString.h | 4 +-- src/Functions/geohashesInBox.cpp | 2 +- src/Functions/isValidUTF8.cpp | 2 +- src/IO/WriteBufferFromHTTPServerResponse.h | 2 +- src/IO/tests/var_uint.cpp | 2 +- .../ASTDictionaryAttributeDeclaration.h | 2 +- src/Parsers/ExpressionElementParsers.cpp | 2 +- src/Parsers/ExpressionElementParsers.h | 2 +- src/Parsers/ParserCreateQuery.h | 2 +- src/Processors/ConcatProcessor.h | 2 +- src/Processors/Executors/ExecutingGraph.h | 2 +- src/Processors/Executors/ThreadsQueue.h | 2 +- src/Processors/Formats/IOutputFormat.h | 2 +- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 2 +- .../Formats/Impl/AvroRowInputFormat.cpp | 2 +- .../Impl/JSONEachRowRowInputFormat.cpp | 6 ++-- src/Processors/IProcessor.h | 10 +++--- .../Algorithms/SummingSortedAlgorithm.cpp | 2 +- src/Processors/Port.h | 2 +- src/Processors/QueryPipeline.cpp | 2 +- src/Processors/ResizeProcessor.h | 4 +-- .../Transforms/ConvertingTransform.h | 2 +- src/Processors/Transforms/FillingTransform.h | 2 +- ...ergingAggregatedMemoryEfficientTransform.h | 8 ++--- src/Server/PostgreSQLHandler.cpp | 2 +- src/Storages/AlterCommands.h | 2 +- src/Storages/CheckResults.h | 2 +- src/Storages/ColumnDependency.h | 2 +- src/Storages/IStorage.cpp | 2 +- src/Storages/IStorage.h | 10 +++--- .../Kafka/ReadBufferFromKafkaConsumer.cpp | 20 ++++++------ .../Kafka/WriteBufferToKafkaProducer.cpp | 2 +- src/Storages/Kafka/parseSyslogLevel.cpp | 2 +- src/Storages/KeyDescription.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 4 +-- src/Storages/MergeTree/IMergeTreeReader.cpp | 2 +- src/Storages/MergeTree/LevelMergeSelector.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 8 ++--- src/Storages/MergeTree/MergeTreeData.h | 4 +-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 ++-- src/Storages/MergeTree/MergeTreePartsMover.h | 2 +- .../MergeTree/MergeTreeRangeReader.cpp | 2 +- ...rgeTreeThreadSelectBlockInputProcessor.cpp | 2 +- .../ReplicatedMergeTreeAltersSequence.h | 2 +- .../ReplicatedMergeTreePartCheckThread.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 6 ++-- .../MergeTree/ReplicatedMergeTreeQueue.h | 4 +-- .../MergeTree/registerStorageMergeTree.cpp | 4 +-- src/Storages/MutationCommands.h | 2 +- src/Storages/StorageInMemoryMetadata.h | 6 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/TTLDescription.h | 2 +- src/TableFunctions/ITableFunctionFileLike.h | 2 +- 132 files changed, 231 insertions(+), 229 deletions(-) diff --git a/src/Access/AllowedClientHosts.cpp b/src/Access/AllowedClientHosts.cpp index 1cee8a2f782..1bcdd26c383 100644 --- a/src/Access/AllowedClientHosts.cpp +++ b/src/Access/AllowedClientHosts.cpp @@ -156,7 +156,7 @@ namespace subnet = IPSubnet{pattern}; } - /// Extracts a subnet, a host name or a host name regular expession from a like pattern. + /// Extracts a subnet, a host name or a host name regular expression from a like pattern. void parseLikePattern( const String & pattern, std::optional & subnet, std::optional & name, std::optional & name_regexp) { diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index 081fed87bd2..76c87d948a0 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -34,7 +34,7 @@ public: template std::vector findAll() const { return findAll(EntityClassT::TYPE); } - /// Searchs for an entity with specified type and name. Returns std::nullopt if not found. + /// Searches for an entity with specified type and name. Returns std::nullopt if not found. std::optional find(EntityType type, const String & name) const; template @@ -45,7 +45,7 @@ public: template std::vector find(const Strings & names) const { return find(EntityClassT::TYPE, names); } - /// Searchs for an entity with specified name and type. Throws an exception if not found. + /// Searches for an entity with specified name and type. Throws an exception if not found. UUID getID(EntityType type, const String & name) const; template diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 76258f5cd7d..d457d01f523 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -509,20 +509,20 @@ public: UInt64 count = 0; if (isSmall()) { - std::vector ans; + std::vector answer; for (const auto & x : small) { T val = x.getValue(); if (UInt32(val) >= range_start) { - ans.push_back(val); + answer.push_back(val); } } - sort(ans.begin(), ans.end()); - if (limit > ans.size()) - limit = ans.size(); + sort(answer.begin(), answer.end()); + if (limit > answer.size()) + limit = answer.size(); for (size_t i = 0; i < limit; ++i) - r1.add(ans[i]); + r1.add(answer[i]); count = UInt64(limit); } else diff --git a/src/AggregateFunctions/AggregateFunctionMLMethod.cpp b/src/AggregateFunctions/AggregateFunctionMLMethod.cpp index d6dce6fe080..413209fd2dc 100644 --- a/src/AggregateFunctions/AggregateFunctionMLMethod.cpp +++ b/src/AggregateFunctions/AggregateFunctionMLMethod.cpp @@ -200,7 +200,7 @@ void LinearModelData::merge(const DB::LinearModelData & rhs) updateState(); /// can't update rhs state because it's constant - /// squared mean is more stable (in sence of quality of prediction) when two states with quietly different number of learning steps are merged + /// squared mean is more stable (in sense of quality of prediction) when two states with quietly different number of learning steps are merged Float64 frac = (static_cast(iter_num) * iter_num) / (iter_num * iter_num + rhs.iter_num * rhs.iter_num); for (size_t i = 0; i < weights.size(); ++i) diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index 2f2c23fdc8b..2f34aef0932 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -28,7 +28,7 @@ namespace ErrorCodes /// If all rows had NULL, the behaviour is determined by "result_is_nullable" template parameter. /// true - return NULL; false - return value from empty aggregation state of nested function. -/// When serialize_flag is set to true, the flag about presense of values is serialized +/// When serialize_flag is set to true, the flag about presence of values is serialized /// regardless to the "result_is_nullable" even if it's unneeded - for protocol compatibility. template diff --git a/src/AggregateFunctions/AggregateFunctionSequenceMatch.h b/src/AggregateFunctions/AggregateFunctionSequenceMatch.h index 79463e890e4..c7e2a700dd7 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceMatch.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceMatch.h @@ -261,7 +261,7 @@ private: if (actions.back().type != PatternActionType::SpecificEvent && actions.back().type != PatternActionType::AnyEvent && actions.back().type != PatternActionType::KleeneStar) - throw Exception{"Temporal condition should be preceeded by an event condition", ErrorCodes::BAD_ARGUMENTS}; + throw Exception{"Temporal condition should be preceded by an event condition", ErrorCodes::BAD_ARGUMENTS}; pattern_has_time = true; actions.emplace_back(type, duration); diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 9d553ebddb4..ebae17a8a07 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -124,7 +124,7 @@ struct SumMapVariants }; // This template gives an aggregate function template that is narrowed -// to accept either tuple argumen or normal argumens. +// to accept either tuple argumen or normal arguments. template struct MinMapDispatchOnTupleArgument { @@ -133,7 +133,7 @@ struct MinMapDispatchOnTupleArgument }; // This template gives an aggregate function template that is narrowed -// to accept either tuple argumen or normal argumens. +// to accept either tuple argumen or normal arguments. template struct MaxMapDispatchOnTupleArgument { diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index 3f41046c20e..3297819a9ff 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -148,7 +148,7 @@ private: // Loop through the entire events_list, update the event timestamp value - // The level path must be 1---2---3---...---check_events_size, find the max event level that statisfied the path in the sliding window. + // The level path must be 1---2---3---...---check_events_size, find the max event level that satisfied the path in the sliding window. // If found, returns the max event level, else return 0. // The Algorithm complexity is O(n). UInt8 getEventLevel(Data & data) const @@ -160,7 +160,7 @@ private: data.sort(); - /// events_timestamp stores the timestamp that latest i-th level event happen withing time window after previous level event. + /// events_timestamp stores the timestamp that latest i-th level event happen within time window after previous level event. /// timestamp defaults to -1, which unsigned timestamp value never meet /// there may be some bugs when UInt64 type timstamp overflows Int64, but it works on most cases. std::vector events_timestamp(events_size, -1); diff --git a/src/AggregateFunctions/IAggregateFunctionCombinator.h b/src/AggregateFunctions/IAggregateFunctionCombinator.h index 89c313567a3..c1ab621c158 100644 --- a/src/AggregateFunctions/IAggregateFunctionCombinator.h +++ b/src/AggregateFunctions/IAggregateFunctionCombinator.h @@ -54,7 +54,7 @@ public: /** Create combined aggregate function (ex: sumIf) * from nested function (ex: sum) - * and arguments for combined agggregate function (ex: UInt64, UInt8 for sumIf). + * and arguments for combined aggregate function (ex: UInt64, UInt8 for sumIf). * It's assumed that function transformArguments was called before this function and 'arguments' are validated. */ virtual AggregateFunctionPtr transformAggregateFunction( diff --git a/src/Client/TimeoutSetter.cpp b/src/Client/TimeoutSetter.cpp index a512485407e..87368f93ba3 100644 --- a/src/Client/TimeoutSetter.cpp +++ b/src/Client/TimeoutSetter.cpp @@ -34,7 +34,7 @@ TimeoutSetter::~TimeoutSetter() } catch (std::exception & e) { - // Sometimes catched on macos + // Sometimes caught on macos LOG_ERROR(&Poco::Logger::get("Client"), "TimeoutSetter: Can't reset timeouts: {}", e.what()); } } diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index de4668e68d4..6c364680174 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -228,7 +228,7 @@ public: using Filter = PaddedPODArray; virtual Ptr filter(const Filter & filt, ssize_t result_size_hint) const = 0; - /// Permutes elements using specified permutation. Is used in sortings. + /// Permutes elements using specified permutation. Is used in sorting. /// limit - if it isn't 0, puts only first limit elements in the result. using Permutation = PaddedPODArray; virtual Ptr permute(const Permutation & perm, size_t limit) const = 0; @@ -239,7 +239,7 @@ public: /** Compares (*this)[n] and rhs[m]. Column rhs should have the same type. * Returns negative number, 0, or positive number (*this)[n] is less, equal, greater than rhs[m] respectively. - * Is used in sortings. + * Is used in sorting. * * If one of element's value is NaN or NULLs, then: * - if nan_direction_hint == -1, NaN and NULLs are considered as least than everything other; diff --git a/src/Common/ArrayCache.h b/src/Common/ArrayCache.h index 5e3af09a2fd..caa26d237bf 100644 --- a/src/Common/ArrayCache.h +++ b/src/Common/ArrayCache.h @@ -557,7 +557,7 @@ public: /// Only one of several concurrent threads calling this method will call get_size or initialize, /// others will wait for that call to complete and will use its result (this helps prevent cache stampede). /// - /// Exceptions occuring in callbacks will be propagated to the caller. + /// Exceptions occurring in callbacks will be propagated to the caller. /// Another thread from the set of concurrent threads will then try to call its callbacks etc. /// /// Returns cached value wrapped by holder, preventing cache entry from eviction. diff --git a/src/Common/AutoArray.h b/src/Common/AutoArray.h index a39706baa3d..1df8bde8c12 100644 --- a/src/Common/AutoArray.h +++ b/src/Common/AutoArray.h @@ -15,7 +15,7 @@ namespace ErrorCodes extern const int CANNOT_ALLOCATE_MEMORY; } -/** An array of (almost) unchangable size: +/** An array of (almost) unchangeable size: * the size is specified in the constructor; * `resize` method removes old data, and necessary only for * so that you can first create an empty object using the default constructor, diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index fb4ae8ec41a..d4a2dfbafe5 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -116,7 +116,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac } config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir); - /** We should remember last modification time if and only if config was sucessfully loaded + /** We should remember last modification time if and only if config was successfully loaded * Otherwise a race condition could occur during config files update: * File is contain raw (and non-valid) data, therefore config is not applied. * When file has been written (and contain valid data), we don't load new data since modification time remains the same. diff --git a/src/Common/Config/ConfigReloader.h b/src/Common/Config/ConfigReloader.h index 553589fbd37..489f062e2fe 100644 --- a/src/Common/Config/ConfigReloader.h +++ b/src/Common/Config/ConfigReloader.h @@ -42,7 +42,7 @@ public: ~ConfigReloader(); - /// Call this method to run the backround thread. + /// Call this method to run the background thread. void start(); /// Reload immediately. For SYSTEM RELOAD CONFIG query. diff --git a/src/Common/CounterInFile.h b/src/Common/CounterInFile.h index 3ede8349a17..48414bd09cc 100644 --- a/src/Common/CounterInFile.h +++ b/src/Common/CounterInFile.h @@ -63,7 +63,7 @@ public: if (file_doesnt_exists && !create_if_need) { throw Poco::Exception("File " + path + " does not exist. " - "You must create it manulally with appropriate value or 0 for first start."); + "You must create it manually with appropriate value or 0 for first start."); } int fd = ::open(path.c_str(), O_RDWR | O_CREAT | O_CLOEXEC, 0666); diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 4bab9ef2844..a82853f9961 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -33,8 +33,8 @@ M(QueryThread, "Number of query processing threads") \ M(ReadonlyReplica, "Number of Replicated tables that are currently in readonly state due to re-initialization after ZooKeeper session loss or due to startup without ZooKeeper configured.") \ M(MemoryTracking, "Total amount of memory (bytes) allocated by the server.") \ - M(MemoryTrackingInBackgroundProcessingPool, "Total amount of memory (bytes) allocated in background processing pool (that is dedicated for backround merges, mutations and fetches). Note that this value may include a drift when the memory was allocated in a context of background processing pool and freed in other context or vice-versa. This happens naturally due to caches for tables indexes and doesn't indicate memory leaks.") \ - M(MemoryTrackingInBackgroundMoveProcessingPool, "Total amount of memory (bytes) allocated in background processing pool (that is dedicated for backround moves). Note that this value may include a drift when the memory was allocated in a context of background processing pool and freed in other context or vice-versa. This happens naturally due to caches for tables indexes and doesn't indicate memory leaks.") \ + M(MemoryTrackingInBackgroundProcessingPool, "Total amount of memory (bytes) allocated in background processing pool (that is dedicated for background merges, mutations and fetches). Note that this value may include a drift when the memory was allocated in a context of background processing pool and freed in other context or vice-versa. This happens naturally due to caches for tables indexes and doesn't indicate memory leaks.") \ + M(MemoryTrackingInBackgroundMoveProcessingPool, "Total amount of memory (bytes) allocated in background processing pool (that is dedicated for background moves). Note that this value may include a drift when the memory was allocated in a context of background processing pool and freed in other context or vice-versa. This happens naturally due to caches for tables indexes and doesn't indicate memory leaks.") \ M(MemoryTrackingInBackgroundSchedulePool, "Total amount of memory (bytes) allocated in background schedule pool (that is dedicated for bookkeeping tasks of Replicated tables).") \ M(MemoryTrackingInBackgroundBufferFlushSchedulePool, "Total amount of memory (bytes) allocated in background buffer flushes pool (that is dedicated for background buffer flushes).") \ M(MemoryTrackingInBackgroundDistributedSchedulePool, "Total amount of memory (bytes) allocated in background distributed schedule pool (that is dedicated for distributed sends).") \ diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index b58429a69d6..876cbd8a66b 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -32,7 +32,7 @@ class InternalTextLogsQueue; class CurrentThread { public: - /// Return true in case of successful initializaiton + /// Return true in case of successful initialization static bool isInitialized(); /// Handler to current thread diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index a3756d71c1d..d61982f3406 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -284,7 +284,7 @@ bool DNSResolver::updateCache() impl->host_name.emplace(updated_host_name); } - /// FIXME Updating may take a long time becouse we cannot manage timeouts of getaddrinfo(...) and getnameinfo(...). + /// FIXME Updating may take a long time because we cannot manage timeouts of getaddrinfo(...) and getnameinfo(...). /// DROP DNS CACHE will wait on update_mutex (possibly while holding drop_mutex) std::lock_guard lock(impl->update_mutex); diff --git a/src/Common/FieldVisitors.h b/src/Common/FieldVisitors.h index a749432500f..3988d946db3 100644 --- a/src/Common/FieldVisitors.h +++ b/src/Common/FieldVisitors.h @@ -99,7 +99,7 @@ public: }; -/** Converts numberic value of any type to specified type. */ +/** Converts numeric value of any type to specified type. */ template class FieldVisitorConvertToNumber : public StaticVisitor { diff --git a/src/Common/HashTable/FixedHashTable.h b/src/Common/HashTable/FixedHashTable.h index 9d18f03a30b..b3fa5b7a26c 100644 --- a/src/Common/HashTable/FixedHashTable.h +++ b/src/Common/HashTable/FixedHashTable.h @@ -29,7 +29,7 @@ struct FixedHashTableCell void setZero() { full = false; } static constexpr bool need_zero_value_storage = false; - /// This Cell is only stored inside an iterator. It's used to accomodate the fact + /// This Cell is only stored inside an iterator. It's used to accommodate the fact /// that the iterator based API always provide a reference to a continuous memory /// containing the Key. As a result, we have to instantiate a real Key field. /// All methods that return a mutable reference to the Key field are named with @@ -92,7 +92,7 @@ struct FixedHashTableCalculatedSize * than a HashTable in that keys are not stored in the Cell buf, but inferred * inside each iterator. There are a bunch of to make it faster than using * HashTable: a) It doesn't have a conflict chain; b) There is no key - * comparision; c) The number of cycles for checking cell empty is halved; d) + * comparison; c) The number of cycles for checking cell empty is halved; d) * Memory layout is tighter, especially the Clearable variants. * * NOTE: For Set variants this should always be better. For Map variants diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 76aa9705a91..d75c8caf1fc 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -67,7 +67,7 @@ public: /// produce it, saves the result in the cache and returns it. /// Only one of several concurrent threads calling getOrSet() will call load_func(), /// others will wait for that call to complete and will use its result (this helps prevent cache stampede). - /// Exceptions occuring in load_func will be propagated to the caller. Another thread from the + /// Exceptions occurring in load_func will be propagated to the caller. Another thread from the /// set of concurrent threads will then try to call its load_func etc. /// /// Returns std::pair of the cached value and a bool indicating whether the value was produced during this call. diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index ca21cec0212..fa11d373071 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -79,7 +79,7 @@ void MemoryTracker::alloc(Int64 size) /// Cap the limit to the total_memory_tracker, since it may include some drift. /// - /// And since total_memory_tracker is reseted to the process resident + /// And since total_memory_tracker is reset to the process resident /// memory peridically (in AsynchronousMetrics::update()), any limit can be /// capped to it, to avoid possible drift. if (unlikely(current_hard_limit && will_be > current_hard_limit)) diff --git a/src/Common/NamePrompter.h b/src/Common/NamePrompter.h index 116e2b43a0a..a52a5f3775e 100644 --- a/src/Common/NamePrompter.h +++ b/src/Common/NamePrompter.h @@ -77,16 +77,16 @@ private: static std::vector release(DistanceIndexQueue & queue, const std::vector & prompting_strings) { - std::vector ans; - ans.reserve(queue.size()); + std::vector answer; + answer.reserve(queue.size()); while (!queue.empty()) { auto top = queue.top(); queue.pop(); - ans.push_back(prompting_strings[top.second]); + answer.push_back(prompting_strings[top.second]); } - std::reverse(ans.begin(), ans.end()); - return ans; + std::reverse(answer.begin(), answer.end()); + return answer; } }; diff --git a/src/Common/ObjectPool.h b/src/Common/ObjectPool.h index 0cb0714a2b9..01768bf1de5 100644 --- a/src/Common/ObjectPool.h +++ b/src/Common/ObjectPool.h @@ -24,7 +24,7 @@ class SimpleObjectPool { protected: - /// Hold all avaiable objects in stack. + /// Hold all available objects in stack. std::mutex mutex; std::stack> stack; diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index 1fddea4507b..b961bf93cfe 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -44,7 +44,7 @@ namespace ErrorCodes * To be more precise - for use in ColumnVector. * It differs from std::vector in that it does not initialize the elements. * - * Made noncopyable so that there are no accidential copies. You can copy the data using `assign` method. + * Made noncopyable so that there are no accidental copies. You can copy the data using `assign` method. * * Only part of the std::vector interface is supported. * diff --git a/src/Common/QueryProfiler.h b/src/Common/QueryProfiler.h index 581d6d75bf2..44eeebbf10a 100644 --- a/src/Common/QueryProfiler.h +++ b/src/Common/QueryProfiler.h @@ -25,7 +25,7 @@ namespace DB * 2. collect thread's current stack trace * 3. write collected stack trace to trace_pipe for TraceCollector * - * Desctructor tries to unset timer and restore previous signal handler. + * Destructor tries to unset timer and restore previous signal handler. * Note that signal handler implementation is defined by template parameter. See QueryProfilerReal and QueryProfilerCpu. */ template diff --git a/src/Common/RWLock.cpp b/src/Common/RWLock.cpp index a8dba490fac..92eda0c21aa 100644 --- a/src/Common/RWLock.cpp +++ b/src/Common/RWLock.cpp @@ -238,7 +238,7 @@ void RWLockImpl::unlock(GroupsContainer::iterator group_it, const String & query { std::lock_guard state_lock(internal_state_mtx); - /// All of theses are Undefined behavior and nothing we can do! + /// All of these are Undefined behavior and nothing we can do! if (rdlock_owner == readers_queue.end() && wrlock_owner == writers_queue.end()) return; if (rdlock_owner != readers_queue.end() && group_it != rdlock_owner) diff --git a/src/Common/SensitiveDataMasker.h b/src/Common/SensitiveDataMasker.h index a6be85d7909..88309459578 100644 --- a/src/Common/SensitiveDataMasker.h +++ b/src/Common/SensitiveDataMasker.h @@ -22,7 +22,7 @@ namespace Util /// It looks like the singleton is the best option here, as /// two users of that object (OwnSplitChannel & Interpreters/executeQuery) -/// can't own/share that Masker properly without syncronization & locks, +/// can't own/share that Masker properly without synchronization & locks, /// and we can't afford setting global locks for each logged line. /// I've considered singleton alternatives, but it's unclear who should own the object, diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 0ffe0f01afc..677af269ca0 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -67,7 +67,7 @@ std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext error << "Invalid address alignment."; break; case BUS_ADRERR: - error << "Non-existant physical address."; + error << "Non-existent physical address."; break; case BUS_OBJERR: error << "Object specific hardware error."; diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index 66260642dcd..ffa7f0462c9 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -33,9 +33,9 @@ But because ClickHouse is linked with most of the symbols exported (-rdynamic fl 3. DWARF debug info. It contains the most detailed information about symbols and everything else. It allows to get source file names and line numbers from addresses. Only available if you use -g option for compiler. It is also used by default for ClickHouse builds, but because of its weight (about two gigabytes) -it is splitted to separate binary and provided in clickhouse-common-static-dbg package. +it is split to separate binary and provided in clickhouse-common-static-dbg package. This separate binary is placed in /usr/lib/debug/usr/bin/clickhouse and is loaded automatically by tools like gdb, addr2line. -When you build ClickHouse by yourself, debug info is not splitted and present in a single huge binary. +When you build ClickHouse by yourself, debug info is not split and present in a single huge binary. What ClickHouse is using to provide good stack traces? @@ -315,7 +315,7 @@ void collectSymbolsFromELF(dl_phdr_info * info, if (ec) return; - /// Debug info and symbol table sections may be splitted to separate binary. + /// Debug info and symbol table sections may be split to separate binary. std::filesystem::path debug_info_path = std::filesystem::path("/usr/lib/debug") / canonical_path.relative_path(); object_name = std::filesystem::exists(debug_info_path) ? debug_info_path : canonical_path; diff --git a/src/Common/TaskStatsInfoGetter.h b/src/Common/TaskStatsInfoGetter.h index 989843a471c..6865c64dc38 100644 --- a/src/Common/TaskStatsInfoGetter.h +++ b/src/Common/TaskStatsInfoGetter.h @@ -18,7 +18,7 @@ public: void getStat(::taskstats & out_stats, pid_t tid) const; - /// Whether the current process has permissions (sudo or cap_net_admin capabilties) to get taskstats info + /// Whether the current process has permissions (sudo or cap_net_admin capabilities) to get taskstats info static bool checkPermissions(); #if defined(OS_LINUX) diff --git a/src/Common/Volnitsky.h b/src/Common/Volnitsky.h index 6d132016337..af97dbdae13 100644 --- a/src/Common/Volnitsky.h +++ b/src/Common/Volnitsky.h @@ -344,7 +344,7 @@ public: auto callback = [this](const VolnitskyTraits::Ngram ngram, const int offset) { return this->putNGramBase(ngram, offset); }; /// ssize_t is used here because unsigned can't be used with condition like `i >= 0`, unsigned always >= 0 - /// And also adding from the end guarantees that we will find first occurence because we will lookup bigger offsets first. + /// And also adding from the end guarantees that we will find first occurrence because we will lookup bigger offsets first. for (auto i = static_cast(needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0; --i) VolnitskyTraits::putNGram(this->needle + i, i + 1, this->needle, callback); } @@ -534,11 +534,11 @@ public: { const size_t fallback_size = fallback_needles.size(); - size_t ans = std::numeric_limits::max(); + size_t answer = std::numeric_limits::max(); for (size_t i = 0; i < fallback_size; ++i) if (fallback_searchers[fallback_needles[i]].search(haystack, haystack_end) != haystack_end) - ans = std::min(ans, fallback_needles[i]); + answer = std::min(answer, fallback_needles[i]); /// check if we have one non empty volnitsky searcher if (step != std::numeric_limits::max()) @@ -554,17 +554,17 @@ public: const auto res = pos - (hash[cell_num].off - 1); const size_t ind = hash[cell_num].id; if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(haystack, haystack_end, res)) - ans = std::min(ans, ind); + answer = std::min(answer, ind); } } } } /* - * if nothing was found, ans + 1 will be equal to zero and we can + * if nothing was found, answer + 1 will be equal to zero and we can * assign it into the result because we need to return the position starting with one */ - return ans + 1; + return answer + 1; } template @@ -572,11 +572,11 @@ public: { const size_t fallback_size = fallback_needles.size(); - UInt64 ans = std::numeric_limits::max(); + UInt64 answer = std::numeric_limits::max(); for (size_t i = 0; i < fallback_size; ++i) if (auto pos = fallback_searchers[fallback_needles[i]].search(haystack, haystack_end); pos != haystack_end) - ans = std::min(ans, pos - haystack); + answer = std::min(answer, pos - haystack); /// check if we have one non empty volnitsky searcher if (step != std::numeric_limits::max()) @@ -592,25 +592,25 @@ public: const auto res = pos - (hash[cell_num].off - 1); const size_t ind = hash[cell_num].id; if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(haystack, haystack_end, res)) - ans = std::min(ans, res - haystack); + answer = std::min(answer, res - haystack); } } } } - if (ans == std::numeric_limits::max()) + if (answer == std::numeric_limits::max()) return 0; - return count_chars(haystack, haystack + ans); + return count_chars(haystack, haystack + answer); } template - inline void searchOneAll(const UInt8 * haystack, const UInt8 * haystack_end, AnsType * ans, const CountCharsCallback & count_chars) const + inline void searchOneAll(const UInt8 * haystack, const UInt8 * haystack_end, AnsType * answer, const CountCharsCallback & count_chars) const { const size_t fallback_size = fallback_needles.size(); for (size_t i = 0; i < fallback_size; ++i) { const UInt8 * ptr = fallback_searchers[fallback_needles[i]].search(haystack, haystack_end); if (ptr != haystack_end) - ans[fallback_needles[i]] = count_chars(haystack, ptr); + answer[fallback_needles[i]] = count_chars(haystack, ptr); } /// check if we have one non empty volnitsky searcher @@ -626,8 +626,10 @@ public: { const auto * res = pos - (hash[cell_num].off - 1); const size_t ind = hash[cell_num].id; - if (ans[ind] == 0 && res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(haystack, haystack_end, res)) - ans[ind] = count_chars(haystack, res); + if (answer[ind] == 0 + && res + needles[ind].size <= haystack_end + && fallback_searchers[ind].compare(haystack, haystack_end, res)) + answer[ind] = count_chars(haystack, res); } } } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 416e40c2da4..b2e49bee346 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -189,7 +189,7 @@ public: using WaitCondition = std::function; /// Wait for the node to disappear or return immediately if it doesn't exist. - /// If condition is speficied, it is used to return early (when condition returns false) + /// If condition is specified, it is used to return early (when condition returns false) /// The function returns true if waited and false if waiting was interrupted by condition. bool waitForDisappear(const std::string & path, const WaitCondition & condition = {}); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 85fc1cd9f79..a3e82612ac8 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -110,7 +110,7 @@ int32_t err \x00\x00\x00\x00 Client sends requests. For example, create persistent node '/hello' with value 'world'. int32_t request_length \x00\x00\x00\x3a -int32_t xid \x5a\xad\x72\x3f Arbitary number. Used for identification of requests/responses. +int32_t xid \x5a\xad\x72\x3f Arbitrary number. Used for identification of requests/responses. libzookeeper uses unix timestamp for first xid and then autoincrement to that value. int32_t op_num \x00\x00\x00\x01 ZOO_CREATE_OP 1 int32_t path_length \x00\x00\x00\x06 diff --git a/src/Common/formatIPv6.h b/src/Common/formatIPv6.h index 2529d6dc796..844aa123b46 100644 --- a/src/Common/formatIPv6.h +++ b/src/Common/formatIPv6.h @@ -32,7 +32,7 @@ void formatIPv6(const unsigned char * src, char *& dst, uint8_t zeroed_tail_byte * In case of failure returns false and doesn't modify buffer pointed by `dst`. * * @param src - input string, expected to be non-null and null-terminated right after the IPv4 string value. - * @param dst - where to put output bytes, expected to be non-null and atleast IPV4_BINARY_LENGTH-long. + * @param dst - where to put output bytes, expected to be non-null and at IPV4_BINARY_LENGTH-long. * @return false if parsing failed, true otherwise. */ inline bool parseIPv4(const char * src, unsigned char * dst) @@ -68,7 +68,7 @@ inline bool parseIPv4(const char * src, unsigned char * dst) * IPV6_BINARY_LENGTH bytes of buffer pointed by `dst`. * * @param src - input string, expected to be non-null and null-terminated right after the IPv6 string value. -* @param dst - where to put output bytes, expected to be non-null and atleast IPV6_BINARY_LENGTH-long. +* @param dst - where to put output bytes, expected to be non-null and at IPV6_BINARY_LENGTH-long. * @return false if parsing failed, true otherwise. */ inline bool parseIPv6(const char * src, unsigned char * dst) diff --git a/src/Common/intExp.h b/src/Common/intExp.h index b38a8afd83b..d0cc3fa3cc0 100644 --- a/src/Common/intExp.h +++ b/src/Common/intExp.h @@ -14,7 +14,7 @@ #endif -/// On overlow, the function returns unspecified value. +/// On overflow, the function returns unspecified value. inline NO_SANITIZE_UNDEFINED uint64_t intExp2(int x) { return 1ULL << x; diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index 71ddbbd92ea..55708261695 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -9,7 +9,7 @@ namespace DB { -/* Transforms string from grep-wildcard-syntax ("{N..M}", "{a,b,c}" as in remote table function and "*", "?") to perl-regexp for using re2 library fo matching +/* Transforrms string from grep-wildcard-syntax ("{N..M}", "{a,b,c}" as in remote table function and "*", "?") to perl-regexp forr using re2 library for matching * with such steps: * 1) search intervals like {0..9} and enums like {abc,xyz,qwe} in {}, replace them by regexp with pipe (expr1|expr2|expr3), * 2) search and replace "*" and "?". diff --git a/src/Common/tests/chaos_sanitizer.cpp b/src/Common/tests/chaos_sanitizer.cpp index 98f28f95b78..05bf16e3638 100644 --- a/src/Common/tests/chaos_sanitizer.cpp +++ b/src/Common/tests/chaos_sanitizer.cpp @@ -10,7 +10,7 @@ #include -/** Prooves that ThreadFuzzer helps to find concurrency bugs. +/** Proves that ThreadFuzzer helps to find concurrency bugs. * * for i in {1..10}; do ./chaos_sanitizer 1000000; done * for i in {1..10}; do THREAD_FUZZER_CPU_TIME_PERIOD_US=1000 THREAD_FUZZER_SLEEP_PROBABILITY=0.1 THREAD_FUZZER_SLEEP_TIME_US=100000 ./chaos_sanitizer 1000000; done diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index faee8f67a2c..a05b5cd7f64 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -44,7 +44,7 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c std::stringstream message; - /// TODO mess up of endianess in error message. + /// TODO mess up of endianness in error message. message << "Checksum doesn't match: corrupted data." " Reference: " + getHexUIntLowercase(expected_checksum.first) + getHexUIntLowercase(expected_checksum.second) + ". Actual: " + getHexUIntLowercase(calculated_checksum.first) + getHexUIntLowercase(calculated_checksum.second) diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index f45fc82a370..bc8a2a4b541 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -291,7 +291,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest) if (sign) { /// It's well defined for unsigned data types. - /// In constrast, it's undefined to do negation of the most negative signed number due to overflow. + /// In contrast, it's undefined to do negation of the most negative signed number due to overflow. double_delta = -double_delta; } } diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp index b9c7a769345..4328480df41 100644 --- a/src/Compression/tests/gtest_compressionCodec.cpp +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -1274,7 +1274,7 @@ INSTANTIATE_TEST_SUITE_P(Gorilla, ) ); -// These 'tests' try to measure performance of encoding and decoding and hence only make sence to be run locally, +// These 'tests' try to measure performance of encoding and decoding and hence only make sense to be run locally, // also they require pretty big data to run against and generating this data slows down startup of unit test process. // So un-comment only at your discretion. diff --git a/src/Core/BlockInfo.h b/src/Core/BlockInfo.h index 55b57aa77ae..886ecd96ef4 100644 --- a/src/Core/BlockInfo.h +++ b/src/Core/BlockInfo.h @@ -45,7 +45,7 @@ struct BlockInfo void read(ReadBuffer & in); }; -/// Block extention to support delayed defaults. AddingDefaultsBlockInputStream uses it to replace missing values with column defaults. +/// Block extension to support delayed defaults. AddingDefaultsBlockInputStream uses it to replace missing values with column defaults. class BlockMissingValues { public: diff --git a/src/Core/DecimalFunctions.h b/src/Core/DecimalFunctions.h index b1f75c860dc..bd2214d204f 100644 --- a/src/Core/DecimalFunctions.h +++ b/src/Core/DecimalFunctions.h @@ -31,7 +31,7 @@ template <> inline Int64 scaleMultiplier(UInt32 scale) { return common::e template <> inline Int128 scaleMultiplier(UInt32 scale) { return common::exp10_i128(scale); } /** Components of DecimalX value: - * whole - represents whole part of decimal, can be negatve or positive. + * whole - represents whole part of decimal, can be negative or positive. * fractional - for fractional part of decimal, always positive. */ template diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 1f755409f2f..e244581c339 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -64,7 +64,7 @@ #define DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE 54405 #define DBMS_MIN_REVISION_WITH_CLIENT_WRITE_INFO 54420 -/// Mininum revision supporting SettingsBinaryFormat::STRINGS. +/// Minimum revision supporting SettingsBinaryFormat::STRINGS. #define DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS 54429 /// Version of ClickHouse TCP protocol. Set to git tag with latest protocol change. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ff137a54381..64f56d35c49 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -154,7 +154,7 @@ class IColumn; \ M(UInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.", 0) \ \ - M(Bool, insert_deduplicate, true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be preformed", 0) \ + M(Bool, insert_deduplicate, true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be performed", 0) \ \ M(UInt64, insert_quorum, 0, "For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled.", 0) \ M(Milliseconds, insert_quorum_timeout, 600000, "", 0) \ diff --git a/src/DataStreams/AddingDefaultsBlockInputStream.cpp b/src/DataStreams/AddingDefaultsBlockInputStream.cpp index 4caf396eb49..e958b579074 100644 --- a/src/DataStreams/AddingDefaultsBlockInputStream.cpp +++ b/src/DataStreams/AddingDefaultsBlockInputStream.cpp @@ -45,7 +45,7 @@ static void checkCalculated(const ColumnWithTypeAndName & col_read, throw Exception("Unexpected defaults count", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); if (!col_read.type->equals(*col_defaults.type)) - throw Exception("Mismach column types while adding defaults", ErrorCodes::TYPE_MISMATCH); + throw Exception("Mismatch column types while adding defaults", ErrorCodes::TYPE_MISMATCH); } static void mixNumberColumns( @@ -151,7 +151,7 @@ Block AddingDefaultsBlockInputStream::readImpl() if (block_missing_values.empty()) return res; - /// res block alredy has all columns values, with default value for type + /// res block already has all columns values, with default value for type /// (not value specified in table). We identify which columns we need to /// recalculate with help of block_missing_values. Block evaluate_block{res}; diff --git a/src/DataStreams/AsynchronousBlockInputStream.h b/src/DataStreams/AsynchronousBlockInputStream.h index b05948faaf6..77aeacf9c13 100644 --- a/src/DataStreams/AsynchronousBlockInputStream.h +++ b/src/DataStreams/AsynchronousBlockInputStream.h @@ -83,7 +83,7 @@ public: { IBlockInputStream::cancel(kill); - /// Wait for some backgroud calculations to be sure, + /// Wait for some background calculations to be sure, /// that after end of stream nothing is being executing. if (started) pool.wait(); diff --git a/src/DataStreams/MergingSortedBlockInputStream.cpp b/src/DataStreams/MergingSortedBlockInputStream.cpp index 434c89803b8..9c9213ef3cc 100644 --- a/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -249,7 +249,7 @@ void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, TSort return; } - /// We have read all data. Ask childs to cancel providing more data. + /// We have read all data. Ask children to cancel providing more data. cancel(false); finished = true; } diff --git a/src/DataStreams/MergingSortedBlockInputStream.h b/src/DataStreams/MergingSortedBlockInputStream.h index d162fcfb066..7f8d594416d 100644 --- a/src/DataStreams/MergingSortedBlockInputStream.h +++ b/src/DataStreams/MergingSortedBlockInputStream.h @@ -21,7 +21,7 @@ class MergingSortedBlockInputStream : public IBlockInputStream { public: /** limit - if isn't 0, then we can produce only first limit rows in sorted order. - * out_row_sources - if isn't nullptr, then at the end of execution it should contain part numbers of each readed row (and needed flag) + * out_row_sources - if isn't nullptr, then at the end of execution it should contain part numbers of each read row (and needed flag) * quiet - don't log profiling info */ MergingSortedBlockInputStream( diff --git a/src/DataStreams/ParallelInputsProcessor.h b/src/DataStreams/ParallelInputsProcessor.h index 326506d28ca..743e4228970 100644 --- a/src/DataStreams/ParallelInputsProcessor.h +++ b/src/DataStreams/ParallelInputsProcessor.h @@ -327,7 +327,7 @@ private: * - the queue (just processed source will be processed the next time later than the rest) * - stack (just processed source will be processed as soon as possible). * - * The stack is better than the queue when you need to do work on reading one source more consequentially, + * The stack is better than the queue when you need to do work on reading one source more consequently, * and theoretically, this allows you to achieve more consequent/consistent reads from the disk. * * But when using the stack, there is a problem with distributed query processing: diff --git a/src/DataStreams/ParallelParsingBlockInputStream.cpp b/src/DataStreams/ParallelParsingBlockInputStream.cpp index f89a6f030c0..050a0d8ef8a 100644 --- a/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -50,7 +50,7 @@ void ParallelParsingBlockInputStream::cancel(bool kill) /* * The format parsers themselves are not being cancelled here, so we'll * have to wait until they process the current block. Given that the - * chunk size is on the order of megabytes, this should't be too long. + * chunk size is on the order of megabytes, this shouldn't be too long. * We can't call IInputFormat->cancel here, because the parser object is * local to the parser thread, and we don't want to introduce any * synchronization between parser threads and the other threads to get @@ -177,7 +177,7 @@ void ParallelParsingBlockInputStream::parserThreadFunction(ThreadGroupStatusPtr unit.block_ext.block_missing_values.clear(); // We don't know how many blocks will be. So we have to read them all - // until an empty block occured. + // until an empty block occurred. Block block; while (!finished && (block = parser->read()) != Block()) { diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 2d2d678bff6..6e29b4fede8 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -35,7 +35,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( addTableLock( storage->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout)); - /// If the "root" table deduplactes blocks, there are no need to make deduplication for children + /// If the "root" table deduplicates blocks, there are no need to make deduplication for children /// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks bool disable_deduplication_for_children = false; if (!context.getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) diff --git a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h index 3e82b546903..0b75cd0ef88 100644 --- a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h +++ b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h @@ -21,7 +21,7 @@ namespace DB * SimpleAggregateFunction(anyLast, LowCardinality(Nullable(String))) * SimpleAggregateFunction(anyLast, IPv4) * - * Technically, a standard IDataType is instanciated and customized with IDataTypeCustomName and DataTypeCustomDesc. + * Technically, a standard IDataType is instantiated and customized with IDataTypeCustomName and DataTypeCustomDesc. */ class DataTypeCustomSimpleAggregateFunction : public IDataTypeCustomName diff --git a/src/DataTypes/DataTypeDateTime64.h b/src/DataTypes/DataTypeDateTime64.h index 7355964a887..8cbeedfb48c 100644 --- a/src/DataTypes/DataTypeDateTime64.h +++ b/src/DataTypes/DataTypeDateTime64.h @@ -57,7 +57,7 @@ public: * * DateTime64 value and scale factor. * * Suitable Transfotm-types are commonly used in Date/DateTime manipulation functions, - * and should implement static (or const) fucntion with following signatures: + * and should implement static (or const) function with following signatures: * R execute(UInt32 whole_value, ... , const TimeZoneImpl &) * OR * R execute(DateTime64 value, Int64 scale_factor, ... , const TimeZoneImpl &) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 73c966c08c3..9344556f43b 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -307,7 +307,7 @@ public: virtual DataTypePtr promoteNumericType() const; /** Directly insert default value into a column. Default implementation use method IColumn::insertDefault. - * This should be overriden if data type default value differs from column default value (example: Enum data types). + * This should be overridden if data type default value differs from column default value (example: Enum data types). */ virtual void insertDefaultInto(IColumn & column) const; diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index e365c73a845..7dd7bb03c20 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -84,8 +84,8 @@ std::string createCommaSeparatedStringFrom(const Names & names) std::string extractTableName(const std::string & nested_name) { - auto splitted = splitName(nested_name); - return splitted.first; + auto split = splitName(nested_name); + return split.first; } @@ -150,10 +150,10 @@ NamesAndTypesList collect(const NamesAndTypesList & names_and_types) bool collected = false; if (const DataTypeArray * type_arr = typeid_cast(name_type.type.get())) { - auto splitted = splitName(name_type.name); - if (!splitted.second.empty()) + auto split = splitName(name_type.name); + if (!split.second.empty()) { - nested[splitted.first].emplace_back(splitted.second, type_arr->getNestedType()); + nested[split.first].emplace_back(split.second, type_arr->getNestedType()); collected = true; } } @@ -184,12 +184,12 @@ void validateArraySizes(const Block & block) if (!typeid_cast(elem.column.get())) throw Exception("Column with Array type is not represented by ColumnArray column: " + elem.column->dumpStructure(), ErrorCodes::ILLEGAL_COLUMN); - auto splitted = splitName(elem.name); + auto split = splitName(elem.name); /// Is it really a column of Nested data structure. - if (!splitted.second.empty()) + if (!split.second.empty()) { - auto [it, inserted] = nested.emplace(splitted.first, i); + auto [it, inserted] = nested.emplace(split.first, i); /// It's not the first column of Nested data structure. if (!inserted) @@ -200,7 +200,7 @@ void validateArraySizes(const Block & block) if (!first_array_column.hasEqualOffsets(another_array_column)) throw Exception("Elements '" + block.getByPosition(it->second).name + "' and '" + elem.name - + "' of Nested data structure '" + splitted.first + + "' of Nested data structure '" + split.first + "' (Array columns) have different array sizes.", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); } } diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 9cd3e09758e..43434081806 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -299,7 +299,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types) } if (num_supported != type_ids.size()) - throw Exception(getExceptionMessagePrefix(types) + " because some of them have no lossless convertion to Decimal", + throw Exception(getExceptionMessagePrefix(types) + " because some of them have no lossless conversion to Decimal", ErrorCodes::NO_COMMON_TYPE); UInt32 max_scale = 0; diff --git a/src/Dictionaries/BucketCache.h b/src/Dictionaries/BucketCache.h index 1e50d448345..381110066a6 100644 --- a/src/Dictionaries/BucketCache.h +++ b/src/Dictionaries/BucketCache.h @@ -33,8 +33,8 @@ struct Int64Hasher /* Class for storing cache index. It consists of two arrays. - The first one is splitted into buckets (each stores 8 elements (cells)) determined by hash of the element key. - The second one is splitted into 4bit numbers, which are positions in bucket for next element write (So cache uses FIFO eviction algorithm inside each bucket). + The first one is split into buckets (each stores 8 elements (cells)) determined by hash of the element key. + The second one is split into 4bit numbers, which are positions in bucket for next element write (So cache uses FIFO eviction algorithm inside each bucket). */ template class BucketCacheIndex diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index 218de240b63..086e41adead 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -333,7 +333,7 @@ private: /// This lock is used for the inner cache state update function lock it for /// write, when it need to update cache state all other functions just - /// readers. Suprisingly this lock is also used for last_exception pointer. + /// readers. Surprisingly this lock is also used for last_exception pointer. mutable std::shared_mutex rw_lock; /// Actual size will be increased to match power of 2 @@ -342,7 +342,7 @@ private: /// all bits to 1 mask (size - 1) (0b1000 - 1 = 0b111) const size_t size_overlap_mask; - /// Max tries to find cell, overlaped with mask: if size = 16 and start_cell=10: will try cells: 10,11,12,13,14,15,0,1,2,3 + /// Max tries to find cell, overlapped with mask: if size = 16 and start_cell=10: will try cells: 10,11,12,13,14,15,0,1,2,3 static constexpr size_t max_collision_length = 10; const size_t zero_cell_idx{getCellIdx(0)}; @@ -377,7 +377,7 @@ private: * they would be passed as a return value of get(), but for Unknown Reasons the dictionaries use a baroque * interface where get() accepts two callback, one that it calls for found values, and one for not found. * - * Now we make it even uglier by doing this from multiple threads. The missing values are retreived from the + * Now we make it even uglier by doing this from multiple threads. The missing values are retrieved from the * dictionary in a background thread, and this thread calls the provided callback. So if you provide the callbacks, * you MUST wait until the background update finishes, or god knows what happens. Unfortunately, we have no * way to check that you did this right, so good luck. @@ -401,7 +401,7 @@ private: std::vector requested_ids; /// It might seem that it is a leak of performance. - /// But aquiring a mutex without contention is rather cheap. + /// But acquiring a mutex without contention is rather cheap. std::mutex callback_mutex; bool can_use_callback{true}; diff --git a/src/Dictionaries/ComplexKeyCacheDictionary.h b/src/Dictionaries/ComplexKeyCacheDictionary.h index 7c9cf6e3c8e..e055d1a6b33 100644 --- a/src/Dictionaries/ComplexKeyCacheDictionary.h +++ b/src/Dictionaries/ComplexKeyCacheDictionary.h @@ -679,7 +679,7 @@ private: /// all bits to 1 mask (size - 1) (0b1000 - 1 = 0b111) const size_t size_overlap_mask; - /// Max tries to find cell, overlaped with mask: if size = 16 and start_cell=10: will try cells: 10,11,12,13,14,15,0,1,2,3 + /// Max tries to find cell, overlapped with mask: if size = 16 and start_cell=10: will try cells: 10,11,12,13,14,15,0,1,2,3 static constexpr size_t max_collision_length = 10; const UInt64 zero_cell_idx{getCellIdx(StringRef{})}; diff --git a/src/Dictionaries/PolygonDictionaryUtils.cpp b/src/Dictionaries/PolygonDictionaryUtils.cpp index 176711d9002..e35016aaef2 100644 --- a/src/Dictionaries/PolygonDictionaryUtils.cpp +++ b/src/Dictionaries/PolygonDictionaryUtils.cpp @@ -161,7 +161,7 @@ void SlabsPolygonIndex::indexBuild(const std::vector & polygons) if (l == n || sorted_x[l] != all_edges[i].l.x() || sorted_x[r] != all_edges[i].r.x()) { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Error occured while building polygon index. Edge {} is [{}, {}] but found [{}, {}]. l = {}, r = {}", + "Error occurred while building polygon index. Edge {} is [{}, {}] but found [{}, {}]. l = {}, r = {}", i, all_edges[i].l.x(), all_edges[i].r.x(), sorted_x[l], sorted_x[r], l, r); } diff --git a/src/Dictionaries/PolygonDictionaryUtils.h b/src/Dictionaries/PolygonDictionaryUtils.h index 83ce2c26944..11ec28502af 100644 --- a/src/Dictionaries/PolygonDictionaryUtils.h +++ b/src/Dictionaries/PolygonDictionaryUtils.h @@ -83,7 +83,7 @@ private: Poco::Logger * log; - /** Sorted distinct coordinates of all vertexes */ + /** Sorted distinct coordinates of all vertices */ std::vector sorted_x; std::vector all_edges; diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 354a28ff16e..74a1b511f6a 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -349,7 +349,7 @@ struct DecimalBinaryOperation } private: - /// there's implicit type convertion here + /// there's implicit type conversion here static NativeResultType apply(NativeResultType a, NativeResultType b) { if constexpr (can_overflow && _check_overflow) diff --git a/src/Functions/FunctionCustomWeekToSomething.h b/src/Functions/FunctionCustomWeekToSomething.h index 3daed746386..c663e83f5f3 100644 --- a/src/Functions/FunctionCustomWeekToSomething.h +++ b/src/Functions/FunctionCustomWeekToSomething.h @@ -51,7 +51,7 @@ public: "Function " + getName() + " supports 1 or 2 or 3 arguments. The 1st argument " "must be of type Date or DateTime. The 2nd argument (optional) must be " - "a constant UInt8 with week mode. The 3nd argument (optional) must be " + "a constant UInt8 with week mode. The 3rd argument (optional) must be " "a constant string with timezone name", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -67,7 +67,7 @@ public: "Function " + getName() + " supports 1 or 2 or 3 arguments. The 1st argument " "must be of type Date or DateTime. The 2nd argument (optional) must be " - "a constant UInt8 with week mode. The 3nd argument (optional) must be " + "a constant UInt8 with week mode. The 3rd argument (optional) must be " "a constant string with timezone name", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (!isString(arguments[2].type)) @@ -75,7 +75,7 @@ public: "Function " + getName() + " supports 1 or 2 or 3 arguments. The 1st argument " "must be of type Date or DateTime. The 2nd argument (optional) must be " - "a constant UInt8 with week mode. The 3nd argument (optional) must be " + "a constant UInt8 with week mode. The 3rd argument (optional) must be " "a constant string with timezone name", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (isDate(arguments[0].type) && std::is_same_v) diff --git a/src/Functions/FunctionDateOrDateTimeAddInterval.h b/src/Functions/FunctionDateOrDateTimeAddInterval.h index 61b8075efc6..92a6efe9fb7 100644 --- a/src/Functions/FunctionDateOrDateTimeAddInterval.h +++ b/src/Functions/FunctionDateOrDateTimeAddInterval.h @@ -426,7 +426,7 @@ public: template using TransformExecuteReturnType = decltype(std::declval().execute(FieldType(), 0, std::declval())); - // Deduces RETURN DataType from INTPUT DataType, based on return type of Transform{}.execute(INPUT_TYPE, UInt64, DateLUTImpl). + // Deduces RETURN DataType from INPUT DataType, based on return type of Transform{}.execute(INPUT_TYPE, UInt64, DateLUTImpl). // e.g. for Transform-type that has execute()-overload with 'UInt16' input and 'UInt32' return, // argument type is expected to be 'Date', and result type is deduced to be 'DateTime'. template diff --git a/src/Functions/FunctionsBitmap.h b/src/Functions/FunctionsBitmap.h index 9e58eb67a1b..ea19fe04a5a 100644 --- a/src/Functions/FunctionsBitmap.h +++ b/src/Functions/FunctionsBitmap.h @@ -33,10 +33,10 @@ namespace ErrorCodes * Convert bitmap to integer array: * bitmapToArray: bitmap -> integer[] * - * Retrun the smallest value in the set: + * Return the smallest value in the set: * bitmapMin: bitmap -> integer * - * Retrun the greatest value in the set: + * Return the greatest value in the set: * bitmapMax: bitmap -> integer * * Return subset in specified range (not include the range_end): @@ -60,7 +60,7 @@ namespace ErrorCodes * Two bitmap andnot calculation: * bitmapAndnot: bitmap,bitmap -> bitmap * - * Retrun bitmap cardinality: + * Return bitmap cardinality: * bitmapCardinality: bitmap -> integer * * Two bitmap and calculation, return cardinality: diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index c2a7f3f3cd2..9ad5620befb 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -378,7 +378,7 @@ struct ConvertImpl) data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss.") + vec_from.getScale() + 1)); else - data_to.resize(size * 3); /// Arbitary + data_to.resize(size * 3); /// Arbitrary offsets_to.resize(size); @@ -579,7 +579,7 @@ struct ConvertThroughParsing if constexpr (std::is_same_v || to_datetime64) { const auto result_type = removeNullable(block.getByPosition(result).type); - // Time zone is already figured out during result type resultion, no need to do it here. + // Time zone is already figured out during result type resolution, no need to do it here. if (const auto dt_col = checkAndGetDataType(result_type.get())) local_time_zone = &dt_col->getTimeZone(); else @@ -967,7 +967,7 @@ public: else if constexpr (std::is_same_v) return createDecimal(38, scale); - throw Exception("Someting wrong with toDecimalNN()", ErrorCodes::LOGICAL_ERROR); + throw Exception("Something wrong with toDecimalNN()", ErrorCodes::LOGICAL_ERROR); } else { @@ -1211,7 +1211,7 @@ public: res = createDecimal(38, scale); if (!res) - throw Exception("Someting wrong with toDecimalNNOrZero() or toDecimalNNOrNull()", ErrorCodes::LOGICAL_ERROR); + throw Exception("Something wrong with toDecimalNNOrZero() or toDecimalNNOrNull()", ErrorCodes::LOGICAL_ERROR); } else res = std::make_shared(); @@ -1380,7 +1380,7 @@ struct ToNumberMonotonicity return {}; } - /// Size of type is shrinked. + /// Size of type is shrunk. if (size_of_from > size_of_to) { /// Function cannot be monotonic on unbounded ranges. diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 770da2c0477..143900077b7 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -484,7 +484,7 @@ struct ImplXxHash32 static auto apply(const char * s, const size_t len) { return XXH32(s, len, 0); } /** * With current implementation with more than 1 arguments it will give the results - * non-reproducable from outside of CH. + * non-reproducible from outside of CH. * * Proper way of combining several input is to use streaming mode of hash function * https://github.com/Cyan4973/xxHash/issues/114#issuecomment-334908566 @@ -507,7 +507,7 @@ struct ImplXxHash64 /* With current implementation with more than 1 arguments it will give the results - non-reproducable from outside of CH. (see comment on ImplXxHash32). + non-reproducible from outside of CH. (see comment on ImplXxHash32). */ static auto combineHashes(UInt64 h1, UInt64 h2) { return CityHash_v1_0_2::Hash128to64(uint128_t(h1, h2)); } diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index 8e12d3aa0a6..ebabd8f3c87 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -232,7 +232,7 @@ public: /* Register new implementation for function. * - * Arch - required instruction set for running the implementation. It's guarantied that no method would + * Arch - required instruction set for running the implementation. It's guaranteed that no method would * be called (even the constructor and static methods) if the processor doesn't support this instruction set. * * FunctionImpl - implementation, should be inherited from template argument FunctionInterface. diff --git a/src/Functions/PolygonUtils.h b/src/Functions/PolygonUtils.h index d2843e3ec6a..88743312304 100644 --- a/src/Functions/PolygonUtils.h +++ b/src/Functions/PolygonUtils.h @@ -179,9 +179,9 @@ private: { inner, /// The cell is completely inside polygon. outer, /// The cell is completely outside of polygon. - singleLine, /// The cell is splitted to inner/outer part by a single line. - pairOfLinesSingleConvexPolygon, /// The cell is splitted to inner/outer part by a polyline of two sections and inner part is convex. - pairOfLinesSingleNonConvexPolygons, /// The cell is splitted to inner/outer part by a polyline of two sections and inner part is non convex. + singleLine, /// The cell is split to inner/outer part by a single line. + pairOfLinesSingleConvexPolygon, /// The cell is split to inner/outer part by a polyline of two sections and inner part is convex. + pairOfLinesSingleNonConvexPolygons, /// The cell is split to inner/outer part by a polyline of two sections and inner part is non convex. pairOfLinesDifferentPolygons, /// The cell is spliited by two lines to three different parts. complexPolygon /// Generic case. }; diff --git a/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h index df7473686fd..ce20dfe2e60 100644 --- a/src/Functions/TargetSpecific.h +++ b/src/Functions/TargetSpecific.h @@ -10,12 +10,12 @@ * checking platform in runtime (see isArchSupported() below). * * If compiler is not gcc/clang or target isn't x86_64 or ENABLE_MULTITARGET_CODE - * was set to OFF in cmake, all code inside these macroses will be removed and + * was set to OFF in cmake, all code inside these macros will be removed and * USE_MUTLITARGE_CODE will be set to 0. Use #if USE_MUTLITARGE_CODE whenever you * use anything from this namespaces. * * For similarities there is a macros DECLARE_DEFAULT_CODE, which wraps code - * into the namespace TargetSpecific::Default but dosn't specify any additional + * into the namespace TargetSpecific::Default but doesn't specify any additional * copile options. Functions and classes inside this macros are available regardless * of USE_MUTLITARGE_CODE. * diff --git a/src/Functions/abtesting.cpp b/src/Functions/abtesting.cpp index 040630b36ac..16d4e1a1a99 100644 --- a/src/Functions/abtesting.cpp +++ b/src/Functions/abtesting.cpp @@ -247,14 +247,14 @@ public: if (const ColumnConst * col_const_arr = checkAndGetColumnConst(block.getByPosition(arguments[2]).column.get())) { if (!col_const_arr) - throw Exception("Thrid argument for function " + getName() + " must be Array of constant strings", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception("Third argument for function " + getName() + " must be Array of constant strings", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); Array src_arr = col_const_arr->getValue(); for (size_t i = 0; i < src_arr.size(); ++i) { if (src_arr[i].getType() != Field::Types::String) - throw Exception("Thrid argument for function " + getName() + " must be Array of constant strings", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception("Third argument for function " + getName() + " must be Array of constant strings", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); variant_names.push_back(src_arr[i].get()); } } @@ -276,7 +276,7 @@ public: } if (variant_names.size() != xs.size() || xs.size() != ys.size()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sizes of arguments doen't match: variant_names: {}, xs: {}, ys: {}", variant_names.size(), xs.size(), ys.size()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sizes of arguments doesn't match: variant_names: {}, xs: {}, ys: {}", variant_names.size(), xs.size(), ys.size()); if (variant_names.size() < 2) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sizes of arguments must be larger than 1. variant_names: {}, xs: {}, ys: {}", variant_names.size(), xs.size(), ys.size()); diff --git a/src/Functions/array/arrayEnumerateRanked.cpp b/src/Functions/array/arrayEnumerateRanked.cpp index dc7bd6d5dea..7c4b755e020 100644 --- a/src/Functions/array/arrayEnumerateRanked.cpp +++ b/src/Functions/array/arrayEnumerateRanked.cpp @@ -88,7 +88,7 @@ ArraysDepths getArraysDepths(const ColumnsWithTypeAndName & arguments) if (clear_depth > max_array_depth) throw Exception("Incorrect arguments for function arrayEnumerateUniqRanked or arrayEnumerateDenseRanked: clear_depth (" - + std::to_string(clear_depth) + ") cant be larger than max_array_depth (" + std::to_string(max_array_depth) + ").", + + std::to_string(clear_depth) + ") can't be larger than max_array_depth (" + std::to_string(max_array_depth) + ").", ErrorCodes::BAD_ARGUMENTS); return {clear_depth, depths, max_array_depth}; diff --git a/src/Functions/array/arrayEnumerateRanked.h b/src/Functions/array/arrayEnumerateRanked.h index d7dc217c936..64c1b205dec 100644 --- a/src/Functions/array/arrayEnumerateRanked.h +++ b/src/Functions/array/arrayEnumerateRanked.h @@ -17,7 +17,7 @@ * This is very unusual function made as a special order for Yandex.Metrica. * * arrayEnumerateUniqRanked(['hello', 'world', 'hello']) = [1, 1, 2] - * - it returns similar structured array containing number of occurence of the corresponding value. + * - it returns similar structured array containing number of occurrence of the corresponding value. * * arrayEnumerateUniqRanked([['hello', 'world'], ['hello'], ['hello']], 1) = [1, 1, 2] * - look at the depth 1 by default. Elements are ['hello', 'world'], ['hello'], ['hello']. diff --git a/src/Functions/extractAllGroups.h b/src/Functions/extractAllGroups.h index 2bbf03276a4..d6ec9fadb73 100644 --- a/src/Functions/extractAllGroups.h +++ b/src/Functions/extractAllGroups.h @@ -31,7 +31,7 @@ enum class ExtractAllGroupsResultKind /** Match all groups of given input string with given re, return array of arrays of matches. * - * Depending on `Impl::Kind`, result is either grouped by grop id (Horizontal) or in order of appearance (Vertical): + * Depending on `Impl::Kind`, result is either grouped by group id (Horizontal) or in order of appearance (Vertical): * * SELECT extractAllGroupsVertical('abc=111, def=222, ghi=333', '("[^"]+"|\\w+)=("[^"]+"|\\w+)') * => diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index 58e441bd42b..b82dada4639 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -61,7 +61,7 @@ template <> struct ActionValueTypeMap { using ActionValueTyp * It is implemented in two steps. * At first step, it creates a pattern of zeros, literal characters, whitespaces, etc. * and quickly fills resulting character array (string column) with this pattern. - * At second step, it walks across the resulting character array and modifies/replaces specific charaters, + * At second step, it walks across the resulting character array and modifies/replaces specific characters, * by calling some functions by pointers and shifting cursor by specified amount. * * Advantages: diff --git a/src/Functions/formatString.h b/src/Functions/formatString.h index 69db4aadc2a..591015f14cb 100644 --- a/src/Functions/formatString.h +++ b/src/Functions/formatString.h @@ -225,7 +225,7 @@ struct FormatImpl index_positions = big_index_positions_buffer.get(); } - /// Vector of substrings of pattern that will be copied to the ans, not string view because of escaping and iterators invalidation. + /// Vector of substrings of pattern that will be copied to the answer, not string view because of escaping and iterators invalidation. /// These are exactly what is between {} tokens, for `Hello {} world {}` we will have [`Hello `, ` world `, ``]. std::vector substrings; @@ -236,7 +236,7 @@ struct FormatImpl for (String & str : substrings) { /// To use memcpySmallAllowReadWriteOverflow15 for substrings we should allocate a bit more to each string. - /// That was chosen due to perfomance issues. + /// That was chosen due to performance issues. if (!str.empty()) str.reserve(str.size() + right_padding); final_size += str.size(); diff --git a/src/Functions/geohashesInBox.cpp b/src/Functions/geohashesInBox.cpp index ebd476c650c..7fc0b00fb57 100644 --- a/src/Functions/geohashesInBox.cpp +++ b/src/Functions/geohashesInBox.cpp @@ -148,7 +148,7 @@ public: if (!res_offsets.empty() && res_offsets.back() != res_strings.size()) { - throw Exception("Arrary column size mismatch (internal logical error)" + + throw Exception("Array column size mismatch (internal logical error)" + std::to_string(res_offsets.back()) + " != " + std::to_string(res_strings.size()), ErrorCodes::LOGICAL_ERROR); } diff --git a/src/Functions/isValidUTF8.cpp b/src/Functions/isValidUTF8.cpp index 52420d5b7bc..d8c5d3cc580 100644 --- a/src/Functions/isValidUTF8.cpp +++ b/src/Functions/isValidUTF8.cpp @@ -230,7 +230,7 @@ SOFTWARE. range = _mm_or_si128(range, _mm_alignr_epi8(tmp1, tmp2, 13)); /* - * Now we have below range indices caluclated + * Now we have below range indices calculated * Correct cases: * - 8 for C0~FF * - 3 for 1st byte after F0~FF diff --git a/src/IO/WriteBufferFromHTTPServerResponse.h b/src/IO/WriteBufferFromHTTPServerResponse.h index ffa36c11c5b..85a81c3dda7 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/src/IO/WriteBufferFromHTTPServerResponse.h @@ -97,7 +97,7 @@ public: bool compress_ = false, /// If true - set Content-Encoding header and compress the result. CompressionMethod compression_method_ = CompressionMethod::None); - /// Writes progess in repeating HTTP headers. + /// Writes progress in repeating HTTP headers. void onProgress(const Progress & progress); /// Send at least HTTP headers if no data has been sent yet. diff --git a/src/IO/tests/var_uint.cpp b/src/IO/tests/var_uint.cpp index 7194b401d6d..8ed81162685 100644 --- a/src/IO/tests/var_uint.cpp +++ b/src/IO/tests/var_uint.cpp @@ -24,7 +24,7 @@ static void parse_trash_string_as_uint_must_fail(const std::string & str) return; } - std::cerr << "Parsing must fail, but finished sucessfully x=" << x; + std::cerr << "Parsing must fail, but finished successfully x=" << x; exit(-1); } diff --git a/src/Parsers/ASTDictionaryAttributeDeclaration.h b/src/Parsers/ASTDictionaryAttributeDeclaration.h index d48ae02026f..b34ebc539ec 100644 --- a/src/Parsers/ASTDictionaryAttributeDeclaration.h +++ b/src/Parsers/ASTDictionaryAttributeDeclaration.h @@ -18,7 +18,7 @@ public: ASTPtr default_value; /// Attribute expression ASTPtr expression; - /// Is atribute mirrored to the parent identifier + /// Is attribute mirrored to the parent identifier bool hierarchical; /// Flag that shows whether the id->attribute image is injective bool injective; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 563dd376ac3..76c9bc0e0ee 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1097,7 +1097,7 @@ const char * ParserAlias::restricted_keywords[] = "ASOF", "SEMI", "ANTI", - "ONLY", /// YQL synonim for ANTI. Note: YQL is the name of one of Yandex proprietary languages, completely unrelated to ClickHouse. + "ONLY", /// YQL synonym for ANTI. Note: YQL is the name of one of Yandex proprietary languages, completely unrelated to ClickHouse. "ON", "USING", "PREWHERE", diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index 238d582f0b0..13e3febcebe 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -350,7 +350,7 @@ protected: }; /** Parser for function with arguments like KEY VALUE (space separated) - * no commas alowed, just space-separated pairs. + * no commas allowed, just space-separated pairs. */ class ParserFunctionWithKeyValueArguments : public IParserBase { diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 1e75a2cd1ca..953f657a04e 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -354,7 +354,7 @@ protected: /// Parses complete dictionary create query. Uses ParserDictionary and /// ParserDictionaryAttributeDeclaration. Produces ASTCreateQuery. -/// CREATE DICTIONAY [IF NOT EXISTS] [db.]name (attrs) PRIMARY KEY key SOURCE(s(params)) LAYOUT(l(params)) LIFETIME([min v1 max] v2) [RANGE(min v1 max v2)] +/// CREATE DICTIONARY [IF NOT EXISTS] [db.]name (attrs) PRIMARY KEY key SOURCE(s(params)) LAYOUT(l(params)) LIFETIME([min v1 max] v2) [RANGE(min v1 max v2)] class ParserCreateDictionaryQuery : public IParserBase { protected: diff --git a/src/Processors/ConcatProcessor.h b/src/Processors/ConcatProcessor.h index 64f9712c69a..4a1fc580411 100644 --- a/src/Processors/ConcatProcessor.h +++ b/src/Processors/ConcatProcessor.h @@ -6,7 +6,7 @@ namespace DB { -/** Has arbitary non zero number of inputs and one output. +/** Has arbitrary non zero number of inputs and one output. * All of them have the same structure. * * Pulls all data from first input, then all data from second input, etc... diff --git a/src/Processors/Executors/ExecutingGraph.h b/src/Processors/Executors/ExecutingGraph.h index 22ae12cb71d..4f61241a726 100644 --- a/src/Processors/Executors/ExecutingGraph.h +++ b/src/Processors/Executors/ExecutingGraph.h @@ -90,7 +90,7 @@ public: /// Ports which have changed their state since last processor->prepare() call. /// They changed when neighbour processors interact with connected ports. - /// Tey will be used as arguments for next processor->prepare() (and will be cleaned after that). + /// They will be used as arguments for next processor->prepare() (and will be cleaned after that). IProcessor::PortNumbers updated_input_ports; IProcessor::PortNumbers updated_output_ports; diff --git a/src/Processors/Executors/ThreadsQueue.h b/src/Processors/Executors/ThreadsQueue.h index 4593ec1b87a..f9ca6a14857 100644 --- a/src/Processors/Executors/ThreadsQueue.h +++ b/src/Processors/Executors/ThreadsQueue.h @@ -9,7 +9,7 @@ namespace ErrorCodes /// Simple struct which stores threads with numbers [0 .. num_threads - 1]. /// Allows to push and pop specified thread, or pop any thread if has. -/// Oll operations (except init) are O(1). No memory allocations after init happen. +/// All operations (except init) are O(1). No memory allocations after init happen. struct ThreadsQueue { void init(size_t num_threads) diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index ae5e4d72d3c..bab746c1772 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -34,7 +34,7 @@ protected: bool finished = false; bool finalized = false; - /// Flush data on each consumed chunk. This is intented for interactive applications to output data as soon as it's ready. + /// Flush data on each consumed chunk. This is intended for interactive applications to output data as soon as it's ready. bool auto_flush = false; RowsBeforeLimitCounterPtr rows_before_limit_counter; diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index a998378125f..bf825e367c8 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -50,7 +50,7 @@ namespace DB {arrow::Type::STRING, "String"}, {arrow::Type::BINARY, "String"}, - // TODO: add other types that are convertable to internal ones: + // TODO: add other types that are convertible to internal ones: // 0. ENUM? // 1. UUID -> String // 2. JSON -> String diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index f81d3c87d1f..d683c472899 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -647,7 +647,7 @@ private: Poco::JSON::Parser parser; auto json_body = parser.parse(*response_body).extract(); auto schema = json_body->getValue("schema"); - LOG_TRACE((&Poco::Logger::get("AvroConfluentRowInputFormat")), "Succesfully fetched schema id = {}\n{}", id, schema); + LOG_TRACE((&Poco::Logger::get("AvroConfluentRowInputFormat")), "Successfully fetched schema id = {}\n{}", id, schema); return avro::compileJsonSchemaFromString(schema); } catch (const Exception &) diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 5dbcf62b4f8..6350db3b211 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -42,10 +42,10 @@ JSONEachRowRowInputFormat::JSONEachRowRowInputFormat( name_map[column_name] = i; /// NOTE You could place names more cache-locally. if (format_settings_.import_nested_json) { - const auto splitted = Nested::splitName(column_name); - if (!splitted.second.empty()) + const auto split = Nested::splitName(column_name); + if (!split.second.empty()) { - const StringRef table_name(column_name.data(), splitted.first.size()); + const StringRef table_name(column_name.data(), split.first.size()); name_map[table_name] = NESTED_FIELD; } } diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index e9148dd5075..cfc67b8db7b 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -70,11 +70,11 @@ using Processors = std::vector; * Limiting transformation. Pulls data from input and passes to output. * When there was enough data, says that it doesn't need data on its input and that data on its output port is finished. * - * Resize. Has arbitary number of inputs and arbitary number of outputs. - * Pulls data from whatever ready input and pushes it to randomly choosed free output. + * Resize. Has arbitrary number of inputs and arbitrary number of outputs. + * Pulls data from whatever ready input and pushes it to randomly chosen free output. * Examples: - * Union - merge data from number of inputs to one output in arbitary order. - * Split - read data from one input and pass it to arbitary output. + * Union - merge data from number of inputs to one output in arbitrary order. + * Split - read data from one input and pass it to arbitrary output. * * Concat. Has many inputs and only one output. Pulls all data from first input until it is exhausted, * then all data from second input, etc. and pushes all data to output. @@ -104,7 +104,7 @@ using Processors = std::vector; * TODO Processor with all its parameters should represent "pure" function on streams of data from its input ports. * It's in question, what kind of "pure" function do we mean. * For example, data streams are considered equal up to order unless ordering properties are stated explicitly. - * Another example: we should support the notion of "arbitary N-th of M substream" of full stream of data. + * Another example: we should support the notion of "arbitrary N-th of M substream" of full stream of data. */ class IProcessor diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index bc0c0fef6dc..080dbc00c2c 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -230,7 +230,7 @@ static SummingSortedAlgorithm::ColumnsDefinition defineColumns( continue; } - /// Are they inside the primary key or partiton key? + /// Are they inside the primary key or partition key? if (isInPrimaryKey(description, column.name, i) || isInPartitionKey(column.name, partition_key_columns)) { def.column_numbers_not_to_aggregate.push_back(i); diff --git a/src/Processors/Port.h b/src/Processors/Port.h index 63fef27e81a..acce2371dea 100644 --- a/src/Processors/Port.h +++ b/src/Processors/Port.h @@ -117,7 +117,7 @@ protected: while (!value.compare_exchange_weak(expected, desired)) desired = getPtr((getUInt(expected) & FLAGS_MASK & (~mask)) | flags | getUInt(data)); - /// It's not very safe. In case of exception after exchange and before assigment we will get leak. + /// It's not very safe. In case of exception after exchange and before assignment we will get leak. /// Don't know how to make it better. data = getPtr(getUInt(expected) & PTR_MASK); diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index c914eab4e29..8795e2dfa26 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -101,7 +101,7 @@ static OutputPort * uniteExtremes(const std::vector & ports, const static OutputPort * uniteTotals(const std::vector & ports, const Block & header, QueryPipeline::ProcessorsContainer & processors) { - /// Calculate totals fro several streams. + /// Calculate totals from several streams. /// Take totals from first sources which has any, skip others. /// ->> Concat -> Limit diff --git a/src/Processors/ResizeProcessor.h b/src/Processors/ResizeProcessor.h index 5c9660af113..f9c188e041a 100644 --- a/src/Processors/ResizeProcessor.h +++ b/src/Processors/ResizeProcessor.h @@ -7,10 +7,10 @@ namespace DB { -/** Has arbitary non zero number of inputs and arbitary non zero number of outputs. +/** Has arbitrary non zero number of inputs and arbitrary non zero number of outputs. * All of them have the same structure. * - * Pulls data from arbitary input (whenever it is ready) and pushes it to arbitary output (whenever is is not full). + * Pulls data from arbitrary input (whenever it is ready) and pushes it to arbitrary output (whenever is is not full). * Doesn't do any heavy calculations. * Doesn't preserve an order of data. * diff --git a/src/Processors/Transforms/ConvertingTransform.h b/src/Processors/Transforms/ConvertingTransform.h index 68a914e8441..c880004a8a3 100644 --- a/src/Processors/Transforms/ConvertingTransform.h +++ b/src/Processors/Transforms/ConvertingTransform.h @@ -45,7 +45,7 @@ private: /// How to construct result block. Position in source block, where to get each column. ColumnNumbers conversion; /// Do not check that constants are same. Use value from result_header. - /// This is needed in case run functions which are constatn in query scope, + /// This is needed in case run functions which are constant in query scope, /// but may return different result being executed remotely, like `now64()` or `randConstant()`. /// In this case we replace constants from remote source to constatns from initiator. bool ignore_constant_values; diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 570ff1f7691..3df981634ca 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -7,7 +7,7 @@ namespace DB { /** Implements modifier WITH FILL of ORDER BY clause. - * It fills gaps in data stream by rows with missing values in columns with set WITH FILL and deafult values in other columns. + * It fills gaps in data stream by rows with missing values in columns with set WITH FILL and default values in other columns. * Optionally FROM, TO and STEP values can be specified. */ class FillingTransform : public ISimpleTransform diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h index 1ed1979c0d4..75bdefc228f 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h @@ -20,7 +20,7 @@ namespace DB * We need to read them and merge them by buckets - keeping only a few buckets from each file in RAM simultaneously. * * 2. Merge aggregation results for distributed query processing. - * Partially aggregated data arrives from different servers, which can be splitted down or not, into 256 buckets, + * Partially aggregated data arrives from different servers, which can be split down or not, into 256 buckets, * and these buckets are passed to us by the network from each server in sequence, one by one. * You should also read and merge by the buckets. * @@ -29,10 +29,10 @@ namespace DB * There are a number of sources. They give out blocks with partially aggregated data. * Each source can return one of the following block sequences: * 1. "unsplitted" block with bucket_num = -1; - * 2. "splitted" (two_level) blocks with bucket_num from 0 to 255; + * 2. "split" (two_level) blocks with bucket_num from 0 to 255; * In both cases, there may also be a block of "overflows" with bucket_num = -1 and is_overflows = true; * - * We start from the convention that splitted blocks are always passed in the order of bucket_num. + * We start from the convention that split blocks are always passed in the order of bucket_num. * That is, if a < b, then the bucket_num = a block goes before bucket_num = b. * This is needed for a memory-efficient merge * - so that you do not need to read the blocks up front, but go all the way up by bucket_num. @@ -41,7 +41,7 @@ namespace DB * The overflow block can be presented in any order relative to other blocks (but it can be only one). * * It is necessary to combine these sequences of blocks and return the result as a sequence with the same properties. - * That is, at the output, if there are "splitted" blocks in the sequence, then they should go in the order of bucket_num. + * That is, at the output, if there are "split" blocks in the sequence, then they should go in the order of bucket_num. * * The merge can be performed using several (merging_threads) threads. * For this, receiving of a set of blocks for the next bucket_num should be done sequentially, diff --git a/src/Server/PostgreSQLHandler.cpp b/src/Server/PostgreSQLHandler.cpp index 0f22822e6ba..2b8591d85d8 100644 --- a/src/Server/PostgreSQLHandler.cpp +++ b/src/Server/PostgreSQLHandler.cpp @@ -80,7 +80,7 @@ void PostgreSQLHandler::run() PostgreSQLProtocol::Messaging::ErrorOrNoticeResponse( PostgreSQLProtocol::Messaging::ErrorOrNoticeResponse::ERROR, "0A000", - "ClickHouse doesn't support exteneded query mechanism"), + "ClickHouse doesn't support extended query mechanism"), true); LOG_ERROR(log, "Client tried to access via extended query protocol"); message_transport->dropMessage(); diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index 6dfe796341f..69b6bdff44d 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -147,7 +147,7 @@ private: public: /// Validate that commands can be applied to metadata. - /// Checks that all columns exist and dependecies between them. + /// Checks that all columns exist and dependencies between them. /// This check is lightweight and base only on metadata. /// More accurate check have to be performed with storage->checkAlterIsPossible. void validate(const StorageInMemoryMetadata & metadata, const Context & context) const; diff --git a/src/Storages/CheckResults.h b/src/Storages/CheckResults.h index 0f895fba3bc..245f580f625 100644 --- a/src/Storages/CheckResults.h +++ b/src/Storages/CheckResults.h @@ -9,7 +9,7 @@ namespace DB /// Result of CHECK TABLE query for single part of table struct CheckResult { - /// Part name for merge tree or file name for simplier tables + /// Part name for merge tree or file name for simpler tables String fs_path; /// Does check passed bool success = false; diff --git a/src/Storages/ColumnDependency.h b/src/Storages/ColumnDependency.h index 99445b14806..3f5290efa35 100644 --- a/src/Storages/ColumnDependency.h +++ b/src/Storages/ColumnDependency.h @@ -9,7 +9,7 @@ namespace DB /// Represents dependency from other column. /// Used to determine, which columns we have to read, if we want to update some other column. -/// Necessary, because table can have some depenendecies, which requires several columns for calculation. +/// Necessary, because table can have some dependencies, which requires several columns for calculation. struct ColumnDependency { enum Kind : UInt8 diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 75ff79f62ce..283f5d4955f 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -24,7 +24,7 @@ namespace ErrorCodes bool IStorage::isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const { - /// Virtual column maybe overriden by real column + /// Virtual column maybe overridden by real column return !metadata_snapshot->getColumns().has(column_name) && getVirtuals().contains(column_name); } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 91c05175f87..49437358951 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -137,12 +137,12 @@ public: public: /// Get mutable version (snapshot) of storage metadata. Metadata object is - /// multiversion, so it can be concurrently chaged, but returned copy can be + /// multiversion, so it can be concurrently changed, but returned copy can be /// used without any locks. StorageInMemoryMetadata getInMemoryMetadata() const { return *metadata.get(); } /// Get immutable version (snapshot) of storage metadata. Metadata object is - /// multiversion, so it can be concurrently chaged, but returned copy can be + /// multiversion, so it can be concurrently changed, but returned copy can be /// used without any locks. StorageMetadataPtr getInMemoryMetadataPtr() const { return metadata.get(); } @@ -162,7 +162,7 @@ public: /// virtual columns must contain virtual columns from underlying table. /// /// User can create columns with the same name as virtual column. After that - /// virtual column will be overriden and inaccessible. + /// virtual column will be overridden and inaccessible. /// /// By default return empty list of columns. virtual NamesAndTypesList getVirtuals() const; @@ -196,7 +196,7 @@ public: /// sure, that we execute only one simultaneous alter. Doesn't affect share lock. TableLockHolder lockForAlter(const String & query_id, const std::chrono::milliseconds & acquire_timeout); - /// Lock table exclusively. This lock must be acuired if you want to be + /// Lock table exclusively. This lock must be acquired if you want to be /// sure, that no other thread (SELECT, merge, ALTER, etc.) doing something /// with table. For example it allows to wait all threads before DROP or /// truncate query. @@ -480,7 +480,7 @@ private: mutable RWLock alter_lock = RWLockImpl::create(); /// Lock required for drop queries. Every thread that want to ensure, that - /// table is not dropped have to tabke this lock for read (lockForShare). + /// table is not dropped have to table this lock for read (lockForShare). /// DROP-like queries take this lock for write (lockExclusively), to be sure /// that all table threads finished. mutable RWLock drop_lock = RWLockImpl::create(); diff --git a/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp b/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp index 2498b719648..4114ab05975 100644 --- a/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp +++ b/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp @@ -38,14 +38,14 @@ ReadBufferFromKafkaConsumer::ReadBufferFromKafkaConsumer( , current(messages.begin()) , topics(_topics) { - // called (synchroniously, during poll) when we enter the consumer group + // called (synchronously, during poll) when we enter the consumer group consumer->set_assignment_callback([this](const cppkafka::TopicPartitionList & topic_partitions) { LOG_TRACE(log, "Topics/partitions assigned: {}", topic_partitions); assignment = topic_partitions; }); - // called (synchroniously, during poll) when we leave the consumer group + // called (synchronously, during poll) when we leave the consumer group consumer->set_revocation_callback([this](const cppkafka::TopicPartitionList & topic_partitions) { // Rebalance is happening now, and now we have a chance to finish the work @@ -193,9 +193,9 @@ void ReadBufferFromKafkaConsumer::commit() // in a controlled manner (i.e. we don't know the offsets to commit then) size_t max_retries = 5; - bool commited = false; + bool committed = false; - while (!commited && max_retries > 0) + while (!committed && max_retries > 0) { try { @@ -204,7 +204,7 @@ void ReadBufferFromKafkaConsumer::commit() // there were not enough replicas available for the __consumer_offsets topic. // also some other temporary issues like client-server connectivity problems are possible consumer->commit(); - commited = true; + committed = true; print_offsets("Committed offset", consumer->get_offsets_committed(consumer->get_assignment())); } catch (const cppkafka::HandleException & e) @@ -214,10 +214,10 @@ void ReadBufferFromKafkaConsumer::commit() --max_retries; } - if (!commited) + if (!committed) { - // TODO: insert atomicity / transactions is needed here (possibility to rollback, ot 2 phase commits) - throw Exception("All commit attempts failed. Last block was already written to target table(s), but was not commited to Kafka.", ErrorCodes::CANNOT_COMMIT_OFFSET); + // TODO: insert atomicity / transactions is needed here (possibility to rollback, on 2 phase commits) + throw Exception("All commit attempts failed. Last block was already written to target table(s), but was not committed to Kafka.", ErrorCodes::CANNOT_COMMIT_OFFSET); } } else @@ -348,7 +348,7 @@ bool ReadBufferFromKafkaConsumer::poll() if (!new_messages.empty()) { // we have polled something just after rebalance. - // we will not use current batch, so we need to return to last commited position + // we will not use current batch, so we need to return to last committed position // otherwise we will continue polling from that position resetToLastCommitted("Rewind last poll after rebalance."); } @@ -430,7 +430,7 @@ size_t ReadBufferFromKafkaConsumer::filterMessageErrors() void ReadBufferFromKafkaConsumer::resetIfStopped() { // we can react on stop only during fetching data - // after block is formed (i.e. during copying data to MV / commiting) we ignore stop attempts + // after block is formed (i.e. during copying data to MV / committing) we ignore stop attempts if (stopped) { stalled_status = CONSUMER_STOPPED; diff --git a/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp b/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp index 2a497be96ed..c6d365ce2fe 100644 --- a/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp +++ b/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp @@ -57,7 +57,7 @@ void WriteBufferToKafkaProducer::countRow(const Columns & columns, size_t curren const std::string & last_chunk = chunks.back(); size_t last_chunk_size = offset(); - // if last character of last chunk is delimeter - we don't need it + // if last character of last chunk is delimiter - we don't need it if (delim && last_chunk[last_chunk_size - 1] == delim) --last_chunk_size; diff --git a/src/Storages/Kafka/parseSyslogLevel.cpp b/src/Storages/Kafka/parseSyslogLevel.cpp index 96a6885111a..f4358a13efe 100644 --- a/src/Storages/Kafka/parseSyslogLevel.cpp +++ b/src/Storages/Kafka/parseSyslogLevel.cpp @@ -1,7 +1,7 @@ #include "parseSyslogLevel.h" #include -/// Must be in a sepearate compilation unit due to macros overlaps: +/// Must be in a separate compilation unit due to macros overlaps: /// - syslog (LOG_DEBUG/...) /// - logger_useful.h (LOG_DEBUG(...)/...) std::pair parseSyslogLevel(const int level) diff --git a/src/Storages/KeyDescription.cpp b/src/Storages/KeyDescription.cpp index a3a192820d0..533736d19ed 100644 --- a/src/Storages/KeyDescription.cpp +++ b/src/Storages/KeyDescription.cpp @@ -55,7 +55,7 @@ KeyDescription & KeyDescription::operator=(const KeyDescription & other) /// additional_column is constant property It should never be lost. if (additional_column.has_value() && !other.additional_column.has_value()) - throw Exception("Wrong key assignment, loosing additional_column", ErrorCodes::LOGICAL_ERROR); + throw Exception("Wrong key assignment, losing additional_column", ErrorCodes::LOGICAL_ERROR); additional_column = other.additional_column; return *this; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 5c32882131d..02687fc0171 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -194,7 +194,7 @@ public: * Possible state transitions: * Temporary -> Precommitted: we are trying to commit a fetched, inserted or merged part to active set * Precommitted -> Outdated: we could not to add a part to active set and doing a rollback (for example it is duplicated part) - * Precommitted -> Commited: we successfully committed a part to active dataset + * Precommitted -> Committed: we successfully committed a part to active dataset * Precommitted -> Outdated: a part was replaced by a covering part or DROP PARTITION * Outdated -> Deleting: a cleaner selected this part for deletion * Deleting -> Outdated: if an ZooKeeper error occurred during the deletion, we will retry deletion @@ -363,7 +363,7 @@ private: /// Reads columns names and types from columns.txt void loadColumns(bool require); - /// If checksums.txt exists, reads files' checksums (and sizes) from it + /// If checksums.txt exists, reads file's checksums (and sizes) from it void loadChecksums(bool require); /// Loads marks index granularity into memory diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 0cc81a7c1c1..5740eef6810 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -170,7 +170,7 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns "got " + toString(res_columns.size()), ErrorCodes::LOGICAL_ERROR); /// Convert columns list to block. - /// TODO: rewrite with columns interface. It wll be possible after changes in ExpressionActions. + /// TODO: rewrite with columns interface. It will be possible after changes in ExpressionActions. auto name_and_type = columns.begin(); for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) { diff --git a/src/Storages/MergeTree/LevelMergeSelector.cpp b/src/Storages/MergeTree/LevelMergeSelector.cpp index 5fedd42e72b..ba7ca0257eb 100644 --- a/src/Storages/MergeTree/LevelMergeSelector.cpp +++ b/src/Storages/MergeTree/LevelMergeSelector.cpp @@ -89,7 +89,7 @@ void selectWithinPartition( if (range_size <= max_total_size_to_merge) estimator.consider(parts.begin() + range_begin, parts.begin() + range_end, range_size); - break; /// Minumum level is enough. + break; /// Minimum level is enough. } if (range_begin == 0) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 06e03e9da5e..1f05476a684 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -375,7 +375,7 @@ void MergeTreeData::checkProperties( if (indices_names.find(index.name) != indices_names.end()) throw Exception( - "Index with name " + backQuote(index.name) + " already exsists", + "Index with name " + backQuote(index.name) + " already exists", ErrorCodes::LOGICAL_ERROR); indices_names.insert(index.name); @@ -629,8 +629,8 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat std::back_inserter(names_intersection)); if (!names_intersection.empty()) - throw Exception("Colums: " + Nested::createCommaSeparatedStringFrom(names_intersection) + - " listed both in colums to sum and in partition key. That is not allowed.", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Columns: " + Nested::createCommaSeparatedStringFrom(names_intersection) + + " listed both in columns to sum and in partition key. That is not allowed.", ErrorCodes::BAD_ARGUMENTS); } } @@ -1037,7 +1037,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts(bool force) if (part.unique() && /// Grab only parts that are not used by anyone (SELECTs for example). ((part_remove_time < now && now - part_remove_time > getSettings()->old_parts_lifetime.totalSeconds()) || force - || isInMemoryPart(part))) /// Remove in-memory parts immediatly to not store excessive data in RAM + || isInMemoryPart(part))) /// Remove in-memory parts immediately to not store excessive data in RAM { parts_to_delete.emplace_back(it); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index a50ea2a939f..24d7aa9a4ad 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -422,7 +422,7 @@ public: /// Renames temporary part to a permanent part and adds it to the parts set. /// It is assumed that the part does not intersect with existing parts. - /// If increment != nullptr, part index is determing using increment. Otherwise part index remains unchanged. + /// If increment != nullptr, part index is determining using increment. Otherwise part index remains unchanged. /// If out_transaction != nullptr, adds the part in the PreCommitted state (the part will be added to the /// active set later with out_transaction->commit()). /// Else, commits the part immediately. @@ -850,7 +850,7 @@ protected: const MergeListEntry * merge_entry); /// If part is assigned to merge or mutation (possibly replicated) - /// Should be overriden by childs, because they can have different + /// Should be overridden by children, because they can have different /// mechanisms for parts locking virtual bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 542d40a2e16..6dc78cf255d 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -508,7 +508,7 @@ public: * - time elapsed for current merge. */ -/// Auxilliary struct that for each merge stage stores its current progress. +/// Auxiliary struct that for each merge stage stores its current progress. /// A stage is: the horizontal stage + a stage for each gathered column (if we are doing a /// Vertical merge) or a mutation of a single part. During a single stage all rows are read. struct MergeStageProgress diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index b2f123190e6..2415ce75e77 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -979,17 +979,17 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( if (direction == 1) { /// Split first few ranges to avoid reading much data. - bool splitted = false; + bool split = false; for (auto range : ranges) { - while (!splitted && range.begin + marks_in_range < range.end) + while (!split && range.begin + marks_in_range < range.end) { new_ranges.emplace_back(range.begin, range.begin + marks_in_range); range.begin += marks_in_range; marks_in_range *= 2; if (marks_in_range > max_marks_in_range) - splitted = true; + split = true; } new_ranges.emplace_back(range.begin, range.end); } diff --git a/src/Storages/MergeTree/MergeTreePartsMover.h b/src/Storages/MergeTree/MergeTreePartsMover.h index 1d3b5fc045d..a1afadec7fa 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.h +++ b/src/Storages/MergeTree/MergeTreePartsMover.h @@ -54,7 +54,7 @@ public: /// Replaces cloned part from detached directory into active data parts set. /// Replacing part changes state to DeleteOnDestroy and will be removed from disk after destructor of - ///IMergeTreeDataPart called. If replacing part doesn't exists or not active (commited) than + ///IMergeTreeDataPart called. If replacing part doesn't exists or not active (committed) than /// cloned part will be removed and log message will be reported. It may happen in case of concurrent /// merge or mutation. void swapClonedPart(const std::shared_ptr & cloned_parts) const; diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 8aebb09b8f4..726e405b3e8 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -877,7 +877,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r /// If there is still a filter, do the filtering now if (result.getFilter()) { - /// filter might be shrinked while columns not + /// filter might be shrunk while columns not const auto * result_filter = result.getFilterOriginal(); filterColumns(result.columns, result_filter->getData()); result.need_filter = true; diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp index 0fdc613d8b5..f57247e39ab 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp @@ -32,7 +32,7 @@ MergeTreeThreadSelectBlockInputProcessor::MergeTreeThreadSelectBlockInputProcess { /// round min_marks_to_read up to nearest multiple of block_size expressed in marks /// If granularity is adaptive it doesn't make sense - /// Maybe it will make sence to add settings `max_block_size_bytes` + /// Maybe it will make sense to add settings `max_block_size_bytes` if (max_block_size_rows && !storage.canUseAdaptiveGranularity()) { size_t fixed_index_granularity = storage.getSettings()->index_granularity; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeAltersSequence.h b/src/Storages/MergeTree/ReplicatedMergeTreeAltersSequence.h index b8f42f2baa6..b675a85581e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeAltersSequence.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeAltersSequence.h @@ -8,7 +8,7 @@ namespace DB { /// ALTERs in StorageReplicatedMergeTree have to be executed sequentially (one /// by one). But ReplicatedMergeTreeQueue execute all entries almost -/// concurrently. The only depency between entries is data parts, but they are +/// concurrently. The only dependency between entries is data parts, but they are /// not suitable in alters case. /// /// This class stores information about current alters in diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 75a3c463061..22cb5ed6e9c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -352,7 +352,7 @@ void ReplicatedMergeTreePartCheckThread::run() if (parts_queue.empty()) { - LOG_ERROR(log, "Someone erased cheking part from parts_queue. This is a bug."); + LOG_ERROR(log, "Someone erased checking part from parts_queue. This is a bug."); } else { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index a3ed86cb2b6..1b19f792679 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -985,7 +985,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( MergeTreeData & data, std::lock_guard & state_lock) const { - /// If our entry produce part which is alredy covered by + /// If our entry produce part which is already covered by /// some other entry which is currently executing, then we can postpone this entry. if (entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::GET_PART @@ -1224,7 +1224,7 @@ bool ReplicatedMergeTreeQueue::processEntry( try { /// We don't have any backoff for failed entries - /// we just count amount of tries for each ot them. + /// we just count amount of tries for each of them. if (func(entry)) removeProcessedEntry(get_zookeeper(), entry); } @@ -1905,7 +1905,7 @@ std::optional> ReplicatedMergeTreeMergePredicate::getDesir if (mutation_status->entry->isAlterMutation()) { /// We want to assign mutations for part which version is bigger - /// than part current version. But it doesn't make sence to assign + /// than part current version. But it doesn't make sense to assign /// more fresh versions of alter-mutations if previous alter still /// not done because alters execute one by one in strict order. if (mutation_version > current_version || !mutation_status->is_done) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index d68833bb380..030ae9cfc05 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -118,7 +118,7 @@ private: /// Note that is_done is not equivalent to parts_to_do.size() == 0 /// (even if parts_to_do.size() == 0 some relevant parts can still commit in the future). - /// Also we can jump over mutation when we dowload mutated part from other replica. + /// Also we can jump over mutation when we download mutated part from other replica. bool is_done = false; String latest_failed_part; @@ -356,7 +356,7 @@ public: /// Part maybe fake (look at ReplicatedMergeTreeMergePredicate). void disableMergesInBlockRange(const String & part_name); - /// Cheks that part is already in virtual parts + /// Checks that part is already in virtual parts bool isVirtualPart(const MergeTreeData::DataPartPtr & data_part) const; /// Check that part isn't in currently generating parts and isn't covered by them and add it to future_parts. diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index b7881694565..cf524196c66 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -551,7 +551,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// column if sorting key will be changed. metadata.sorting_key = KeyDescription::getSortingKeyFromAST(args.storage_def->order_by->ptr(), metadata.columns, args.context, merging_param_key_arg); - /// If primary key explicitely defined, than get it from AST + /// If primary key explicitly defined, than get it from AST if (args.storage_def->primary_key) { metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.context); @@ -626,7 +626,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// In old syntax primary_key always equals to sorting key. metadata.primary_key = KeyDescription::getKeyFromAST(engine_args[arg_num], metadata.columns, args.context); - /// But it's not explicitely defined, so we evaluate definition to + /// But it's not explicitly defined, so we evaluate definition to /// nullptr metadata.primary_key.definition_ast = nullptr; diff --git a/src/Storages/MutationCommands.h b/src/Storages/MutationCommands.h index 50541d5c58a..6e641e42cff 100644 --- a/src/Storages/MutationCommands.h +++ b/src/Storages/MutationCommands.h @@ -17,7 +17,7 @@ class WriteBuffer; class ReadBuffer; /// Represents set of actions which should be applied -/// to values from set of columns which statisfy predicate. +/// to values from set of columns which satisfy predicate. struct MutationCommand { ASTPtr ast; /// The AST of the whole command diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 8611d3d6c23..4c78d72a9d1 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -73,7 +73,7 @@ struct StorageInMemoryMetadata /// Set common table TTLs void setTableTTLs(const TTLTableDescription & table_ttl_); - /// TTLs for seperate columns + /// TTLs for separate columns void setColumnTTLs(const TTLColumnsDescription & column_ttls_by_name_); /// Set settings changes in metadata (some settings exlicetely specified in @@ -125,13 +125,13 @@ struct StorageInMemoryMetadata Block getSampleBlockNonMaterialized() const; /// Block with ordinary + materialized + virtuals. Virtuals have to be - /// explicitely specified, because they are part of Storage type, not + /// explicitly specified, because they are part of Storage type, not /// Storage metadata. Block getSampleBlockWithVirtuals(const NamesAndTypesList & virtuals) const; /// Block with ordinary + materialized + aliases + virtuals. Virtuals have - /// to be explicitely specified, because they are part of Storage type, not + /// to be explicitly specified, because they are part of Storage type, not /// Storage metadata. StorageID required only for more clear exception /// message. Block getSampleBlockForColumns( diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 59e3ca4cdc8..75602af218a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3279,7 +3279,7 @@ void StorageReplicatedMergeTree::startup() /// In this thread replica will be activated. restarting_thread.start(); - /// Wait while restarting_thread initializes LeaderElection (and so on) or makes first attmept to do it + /// Wait while restarting_thread initializes LeaderElection (and so on) or makes first attempt to do it startup_event.wait(); /// If we don't separate create/start steps, race condition will happen diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 84fbae73fa5..6c85f0ed7fb 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -379,7 +379,7 @@ private: /// Do the merge or recommend to make the fetch instead of the merge bool tryExecuteMerge(const LogEntry & entry); - /// Execute alter of table metadata. Set replica/metdata and replica/columns + /// Execute alter of table metadata. Set replica/metadata and replica/columns /// nodes in zookeeper and also changes in memory metadata. /// New metadata and columns values stored in entry. bool executeMetadataAlter(const LogEntry & entry); diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index ab93967f114..f7769fd42e9 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -43,7 +43,7 @@ struct TTLDescription /// ^~~~~~~~~~~~~~~~~~~^ ASTPtr expression_ast; - /// Expresion actions evaluated from AST + /// Expression actions evaluated from AST ExpressionActionsPtr expression; /// Result column of this TTL expression diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index 55df9a7484d..101f4104cb8 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -8,7 +8,7 @@ class ColumnsDescription; class Context; /* - * function(source, format, structure) - creates a temporary storage from formated source + * function(source, format, structure) - creates a temporary storage from formatted source */ class ITableFunctionFileLike : public ITableFunction { From 12f66fa82c07fdde4a2410d6aadd0835445b1b87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 04:01:47 +0300 Subject: [PATCH 174/374] Fix 99% of typos --- src/Common/formatIPv6.h | 2 +- src/Common/parseGlobs.cpp | 2 +- src/Interpreters/ArrayJoinedColumnsVisitor.h | 10 +++++----- src/Interpreters/Context.cpp | 2 +- src/Interpreters/Context.h | 4 ++-- src/Interpreters/ConvertStringsToEnumVisitor.h | 2 +- src/Interpreters/DDLWorker.cpp | 8 ++++---- src/Interpreters/ExpressionActions.h | 2 +- .../ExternalLoaderDatabaseConfigRepository.h | 2 +- src/Interpreters/ExternalLoaderXMLConfigRepository.h | 2 +- src/Interpreters/HashJoin.cpp | 2 +- src/Interpreters/HashJoin.h | 2 +- src/Interpreters/IdentifierSemantic.h | 2 +- src/Interpreters/InJoinSubqueriesPreprocessor.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Interpreters/JoinedTables.h | 2 +- src/Interpreters/MergeJoin.cpp | 2 +- src/Interpreters/MergeJoin.h | 2 +- src/Interpreters/MonotonicityCheckVisitor.h | 2 +- src/Interpreters/MutationsInterpreter.cpp | 2 +- src/Interpreters/RemoveInjectiveFunctionsVisitor.h | 2 +- src/Interpreters/Set.cpp | 2 +- src/Interpreters/TreeOptimizer.cpp | 2 +- src/Interpreters/TreeRewriter.cpp | 6 +++--- src/Interpreters/tests/internal_iotop.cpp | 2 +- .../Distributed/DistributedBlockOutputStream.h | 2 +- src/Storages/LiveView/LiveViewBlockInputStream.h | 4 ++-- src/Storages/LiveView/LiveViewEventsBlockInputStream.h | 4 ++-- .../MergeTree/MergeTreeDataPartWriterOnDisk.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexFullText.cpp | 4 ++-- .../MergeTreeThreadSelectBlockInputProcessor.h | 2 +- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp | 2 +- src/Storages/tests/gtest_SplitTokenExtractor.cpp | 2 +- 33 files changed, 46 insertions(+), 46 deletions(-) diff --git a/src/Common/formatIPv6.h b/src/Common/formatIPv6.h index 844aa123b46..63c064b21f8 100644 --- a/src/Common/formatIPv6.h +++ b/src/Common/formatIPv6.h @@ -177,7 +177,7 @@ inline bool parseIPv6(const char * src, unsigned char * dst) } /** Format 4-byte binary sequesnce as IPv4 text: 'aaa.bbb.ccc.ddd', - * expects inout to be in BE-format, that is 0x7f000001 => "127.0.0.1". + * expects in out to be in BE-format, that is 0x7f000001 => "127.0.0.1". * * Any number of the tail bytes can be masked with given mask string. * diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index 55708261695..c07664c4662 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -9,7 +9,7 @@ namespace DB { -/* Transforrms string from grep-wildcard-syntax ("{N..M}", "{a,b,c}" as in remote table function and "*", "?") to perl-regexp forr using re2 library for matching +/* Transforms string from grep-wildcard-syntax ("{N..M}", "{a,b,c}" as in remote table function and "*", "?") to perl-regexp for using re2 library for matching * with such steps: * 1) search intervals like {0..9} and enums like {abc,xyz,qwe} in {}, replace them by regexp with pipe (expr1|expr2|expr3), * 2) search and replace "*" and "?". diff --git a/src/Interpreters/ArrayJoinedColumnsVisitor.h b/src/Interpreters/ArrayJoinedColumnsVisitor.h index d7c40278818..56832914b80 100644 --- a/src/Interpreters/ArrayJoinedColumnsVisitor.h +++ b/src/Interpreters/ArrayJoinedColumnsVisitor.h @@ -98,18 +98,18 @@ private: if (!IdentifierSemantic::getColumnName(node)) return; - auto splitted = Nested::splitName(node.name); /// ParsedParams, Key1 + auto split = Nested::splitName(node.name); /// ParsedParams, Key1 if (array_join_alias_to_name.count(node.name)) { /// ARRAY JOIN was written with an array column. Example: SELECT K1 FROM ... ARRAY JOIN ParsedParams.Key1 AS K1 array_join_result_to_source[node.name] = array_join_alias_to_name[node.name]; /// K1 -> ParsedParams.Key1 } - else if (array_join_alias_to_name.count(splitted.first) && !splitted.second.empty()) + else if (array_join_alias_to_name.count(split.first) && !split.second.empty()) { /// ARRAY JOIN was written with a nested table. Example: SELECT PP.KEY1 FROM ... ARRAY JOIN ParsedParams AS PP array_join_result_to_source[node.name] /// PP.Key1 -> ParsedParams.Key1 - = Nested::concatenateName(array_join_alias_to_name[splitted.first], splitted.second); + = Nested::concatenateName(array_join_alias_to_name[split.first], split.second); } else if (array_join_name_to_alias.count(node.name)) { @@ -119,12 +119,12 @@ private: array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1 array_join_name_to_alias[node.name]] = node.name; } - else if (array_join_name_to_alias.count(splitted.first) && !splitted.second.empty()) + else if (array_join_name_to_alias.count(split.first) && !split.second.empty()) { /** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams AS PP. */ array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1 - Nested::concatenateName(array_join_name_to_alias[splitted.first], splitted.second)] = node.name; + Nested::concatenateName(array_join_name_to_alias[split.first], split.second)] = node.name; } } }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 01097a09cbb..0554607cd7f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1796,7 +1796,7 @@ void Context::updateStorageConfiguration(const Poco::Util::AbstractConfiguration } catch (Exception & e) { - LOG_ERROR(shared->log, "An error has occured while reloading storage policies, storage policies were not applied: {}", e.message()); + LOG_ERROR(shared->log, "An error has occurred while reloading storage policies, storage policies were not applied: {}", e.message()); } } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index dfe2d00873f..c8d13baa9ae 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -532,8 +532,8 @@ public: std::shared_ptr getMetricLog(); std::shared_ptr getAsynchronousMetricLog(); - /// Returns an object used to log opertaions with parts if it possible. - /// Provide table name to make required cheks. + /// Returns an object used to log operations with parts if it possible. + /// Provide table name to make required checks. std::shared_ptr getPartLog(const String & part_database); const MergeTreeSettings & getMergeTreeSettings() const; diff --git a/src/Interpreters/ConvertStringsToEnumVisitor.h b/src/Interpreters/ConvertStringsToEnumVisitor.h index 7e7e15026a8..1e5d8fa9a9b 100644 --- a/src/Interpreters/ConvertStringsToEnumVisitor.h +++ b/src/Interpreters/ConvertStringsToEnumVisitor.h @@ -162,7 +162,7 @@ struct ConvertStringsToEnumMatcher return; /// We are not sure we could change the type of function result - /// cause it is present in other fucntion as argument + /// cause it is present in other function as argument if (data.used_functions.count(function_node.tryGetAlias())) return; diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 2278c0e452f..159d9255adb 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -688,7 +688,7 @@ void DDLWorker::processTask(DDLTask & task, const ZooKeeperPtr & zookeeper) task.execution_status = ExecutionStatus::fromCurrentException("An error occurred before execution"); } - /// We need to distinguish ZK errors occured before and after query executing + /// We need to distinguish ZK errors occurred before and after query executing task.was_executed = true; } @@ -902,8 +902,8 @@ void DDLWorker::cleanupQueue(Int64 current_time_seconds, const ZooKeeperPtr & zo /// Deleting { - Strings childs = zookeeper->getChildren(node_path); - for (const String & child : childs) + Strings children = zookeeper->getChildren(node_path); + for (const String & child : children) { if (child != "lock") zookeeper->tryRemoveRecursive(node_path + "/" + child); @@ -920,7 +920,7 @@ void DDLWorker::cleanupQueue(Int64 current_time_seconds, const ZooKeeperPtr & zo } catch (...) { - LOG_INFO(log, "An error occured while checking and cleaning task {} from queue: {}", node_name, getCurrentExceptionMessage(false)); + LOG_INFO(log, "An error occurred while checking and cleaning task {} from queue: {}", node_name, getCurrentExceptionMessage(false)); } } } diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 49da9a5f810..3697dc02ad3 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -138,7 +138,7 @@ private: void prepare(Block & sample_block, const Settings & settings, NameSet & names_not_for_constant_folding); void executeOnTotals(Block & block) const; - /// Executes action on block (modify it). Block could be splitted in case of JOIN. Then not_processed block is created. + /// Executes action on block (modify it). Block could be split in case of JOIN. Then not_processed block is created. void execute(Block & block, ExtraBlockPtr & not_processed) const; void execute(Block & block, bool dry_run) const; }; diff --git a/src/Interpreters/ExternalLoaderDatabaseConfigRepository.h b/src/Interpreters/ExternalLoaderDatabaseConfigRepository.h index 79bd0009b03..59dad1274e0 100644 --- a/src/Interpreters/ExternalLoaderDatabaseConfigRepository.h +++ b/src/Interpreters/ExternalLoaderDatabaseConfigRepository.h @@ -8,7 +8,7 @@ namespace DB { /// Repository from database, which stores dictionary definitions on disk. -/// Tracks update time and existance of .sql files through IDatabase. +/// Tracks update time and existence of .sql files through IDatabase. class ExternalLoaderDatabaseConfigRepository : public IExternalLoaderConfigRepository { public: diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.h b/src/Interpreters/ExternalLoaderXMLConfigRepository.h index 75d9be8bb42..b89bc06e196 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.h +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.h @@ -35,7 +35,7 @@ private: /// Main server config (config.xml). const Poco::Util::AbstractConfiguration & main_config; - /// Key which contains path to dicrectory with .xml configs for entries + /// Key which contains path to directory with .xml configs for entries std::string config_key; }; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index a807a9fa4ee..b66f1b52376 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -482,7 +482,7 @@ namespace { case HashJoin::Type::EMPTY: break; case HashJoin::Type::CROSS: break; /// Do nothing. We have already saved block, and it is enough. - case HashJoin::Type::DICT: break; /// Noone should call it with Type::DICT. + case HashJoin::Type::DICT: break; /// No one should call it with Type::DICT. #define M(TYPE) \ case HashJoin::Type::TYPE: \ diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index fb879e2c507..36f8354996a 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -88,7 +88,7 @@ using MappedAsof = WithFlags; * - CROSS * * ALL means usual JOIN, when rows are multiplied by number of matching rows from the "right" table. - * ANY uses one line per unique key from right talbe. For LEFT JOIN it would be any row (with needed joined key) from the right table, + * ANY uses one line per unique key from right table. For LEFT JOIN it would be any row (with needed joined key) from the right table, * for RIGHT JOIN it would be any row from the left table and for INNER one it would be any row from right and any row from left. * SEMI JOIN filter left table by keys that are present in right table for LEFT JOIN, and filter right table by keys from left table * for RIGHT JOIN. In other words SEMI JOIN returns only rows which joining keys present in another table. diff --git a/src/Interpreters/IdentifierSemantic.h b/src/Interpreters/IdentifierSemantic.h index 0aef297c734..ca5a923c2ea 100644 --- a/src/Interpreters/IdentifierSemantic.h +++ b/src/Interpreters/IdentifierSemantic.h @@ -16,7 +16,7 @@ struct IdentifierSemanticImpl std::optional membership; /// table position in join }; -/// Static calss to manipulate IdentifierSemanticImpl via ASTIdentifier +/// Static class to manipulate IdentifierSemanticImpl via ASTIdentifier struct IdentifierSemantic { enum class ColumnMatch diff --git a/src/Interpreters/InJoinSubqueriesPreprocessor.h b/src/Interpreters/InJoinSubqueriesPreprocessor.h index 5aa9cfbcadf..8d2e441e05f 100644 --- a/src/Interpreters/InJoinSubqueriesPreprocessor.h +++ b/src/Interpreters/InJoinSubqueriesPreprocessor.h @@ -41,7 +41,7 @@ public: { using Ptr = std::unique_ptr; - /// These methods could be overriden for the need of the unit test. + /// These methods could be overridden for the need of the unit test. virtual bool hasAtLeastTwoShards(const IStorage & table) const; virtual std::pair getRemoteDatabaseAndTableName(const IStorage & table) const; virtual ~CheckShardsAndTables() {} diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index cd2e45323d1..b2f6f57e70b 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -339,7 +339,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( if (col_decl.null_modifier) { if (column_type->isNullable()) - throw Exception("Cant use [NOT] NULL modifier with Nullable type", ErrorCodes::ILLEGAL_SYNTAX_FOR_DATA_TYPE); + throw Exception("Can't use [NOT] NULL modifier with Nullable type", ErrorCodes::ILLEGAL_SYNTAX_FOR_DATA_TYPE); if (*col_decl.null_modifier) column_type = makeNullable(column_type); } diff --git a/src/Interpreters/JoinedTables.h b/src/Interpreters/JoinedTables.h index 3819ec44d41..1e787ee4a65 100644 --- a/src/Interpreters/JoinedTables.h +++ b/src/Interpreters/JoinedTables.h @@ -17,7 +17,7 @@ struct StorageInMemoryMetadata; using StorageMetadataPtr = std::shared_ptr; /// Joined tables' columns resolver. -/// We want to get each table structure at most once per table occurance. Or even better once per table. +/// We want to get each table structure at most once per table occurrence. Or even better once per table. /// TODO: joins tree with costs to change joins order by CBO. class JoinedTables { diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 63e7aa36535..9bf039318d5 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -515,7 +515,7 @@ void MergeJoin::mergeInMemoryRightBlocks() QueryPipeline pipeline; pipeline.init(std::move(source)); - /// TODO: there should be no splitted keys by blocks for RIGHT|FULL JOIN + /// TODO: there should be no split keys by blocks for RIGHT|FULL JOIN pipeline.addPipe({std::make_shared(pipeline.getHeader(), right_sort_description, max_rows_in_right_block, 0, 0, 0, nullptr, 0)}); auto sorted_input = PipelineExecutingBlockInputStream(std::move(pipeline)); diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index 3f88bf7a18f..a13d0304907 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -57,7 +57,7 @@ private: }; /// There're two size limits for right-hand table: max_rows_in_join, max_bytes_in_join. - /// max_bytes is prefered. If it isn't set we approximate it as (max_rows * bytes/row). + /// max_bytes is preferred. If it isn't set we approximate it as (max_rows * bytes/row). struct BlockByteWeight { size_t operator()(const Block & block) const { return block.bytes(); } diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index d813f9618e1..21c63bd58e1 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -86,7 +86,7 @@ public: if (data.isRejected()) return; - /// TODO: monotonicity for fucntions of several arguments + /// TODO: monotonicity for functions of several arguments auto arguments = ast_function.arguments; if (arguments->children.size() != 1) { diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 759543950de..0d3ba60640b 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -480,7 +480,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) /// We care about affected indices because we also need to rewrite them /// when one of index columns updated or filtered with delete. - /// The same about colums, that are needed for calculation of TTL expressions. + /// The same about columns, that are needed for calculation of TTL expressions. if (!dependencies.empty()) { NameSet changed_columns; diff --git a/src/Interpreters/RemoveInjectiveFunctionsVisitor.h b/src/Interpreters/RemoveInjectiveFunctionsVisitor.h index e48bf252659..1adde0d35b0 100644 --- a/src/Interpreters/RemoveInjectiveFunctionsVisitor.h +++ b/src/Interpreters/RemoveInjectiveFunctionsVisitor.h @@ -8,7 +8,7 @@ namespace DB class ASTFunction; -/// Removes unneded injective functions inside `uniq*()`. +/// Removes unneeded injective functions inside `uniq*()`. class RemoveInjectiveFunctionsMatcher { public: diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 19f67bea42c..0f976066165 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -495,7 +495,7 @@ BoolMask MergeTreeSetIndex::checkInRange(const std::vector & key_ranges, }; /** Because each hyperrectangle maps to a contiguous sequence of elements - * layed out in the lexicographically increasing order, the set intersects the range + * laid out in the lexicographically increasing order, the set intersects the range * if and only if either bound coincides with an element or at least one element * is between the lower bounds */ diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 3fdefb3d773..ae0a435edc2 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -360,7 +360,7 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, const C } } -/// If ORDER BY has argument x followed by f(x) transfroms it to ORDER BY x. +/// If ORDER BY has argument x followed by f(x) transforms it to ORDER BY x. /// Optimize ORDER BY x, y, f(x), g(x, y), f(h(x)), t(f(x), g(x)) into ORDER BY x, y /// in case if f(), g(), h(), t() are deterministic (in scope of query). /// Don't optimize ORDER BY f(x), g(x), x even if f(x) is bijection for x or g(x). diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 991742575b8..0b2f8ac3eb7 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -258,10 +258,10 @@ void getArrayJoinedColumns(ASTPtr & query, TreeRewriterResult & result, const AS bool found = false; for (const auto & column : source_columns) { - auto splitted = Nested::splitName(column.name); - if (splitted.first == source_name && !splitted.second.empty()) + auto split = Nested::splitName(column.name); + if (split.first == source_name && !split.second.empty()) { - result.array_join_result_to_source[Nested::concatenateName(result_name, splitted.second)] = column.name; + result.array_join_result_to_source[Nested::concatenateName(result_name, split.second)] = column.name; found = true; break; } diff --git a/src/Interpreters/tests/internal_iotop.cpp b/src/Interpreters/tests/internal_iotop.cpp index 487390c8321..3d17ca32f3f 100644 --- a/src/Interpreters/tests/internal_iotop.cpp +++ b/src/Interpreters/tests/internal_iotop.cpp @@ -54,7 +54,7 @@ static void do_io(size_t id) get_info.getStat(stat, tid); { std::lock_guard lock(mutex); - std::cerr << "#" << id << ", tid " << tid << ", intitial\n" << stat << "\n"; + std::cerr << "#" << id << ", tid " << tid << ", initial\n" << stat << "\n"; } size_t copy_size = 1048576 * (1 + id); diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.h b/src/Storages/Distributed/DistributedBlockOutputStream.h index 53d71ffc424..872776f0867 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.h +++ b/src/Storages/Distributed/DistributedBlockOutputStream.h @@ -30,7 +30,7 @@ class StorageDistributed; * If the Distributed table uses more than one shard, then in order to support the write, * when creating the table, an additional parameter must be specified for ENGINE - the sharding key. * Sharding key is an arbitrary expression from the columns. For example, rand() or UserID. - * When writing, the data block is splitted by the remainder of the division of the sharding key by the total weight of the shards, + * When writing, the data block is split by the remainder of the division of the sharding key by the total weight of the shards, * and the resulting blocks are written in a compressed Native format in separate directories for sending. * For each destination address (each directory with data to send), a separate thread is created in StorageDistributed, * which monitors the directory and sends data. */ diff --git a/src/Storages/LiveView/LiveViewBlockInputStream.h b/src/Storages/LiveView/LiveViewBlockInputStream.h index 9115c6330df..7cab2cb41ed 100644 --- a/src/Storages/LiveView/LiveViewBlockInputStream.h +++ b/src/Storages/LiveView/LiveViewBlockInputStream.h @@ -7,7 +7,7 @@ namespace DB { /** Implements LIVE VIEW table WATCH input stream. - * Keeps stream alive by outputing blocks with no rows + * Keeps stream alive by outputting blocks with no rows * based on period specified by the heartbeat interval. */ class LiveViewBlockInputStream : public IBlockInputStream @@ -127,7 +127,7 @@ protected: if (!active) return { Block(), false }; /// If we are done iterating over our blocks - /// and there are new blocks availble then get them + /// and there are new blocks available then get them if (blocks.get() != (*blocks_ptr).get()) { blocks = (*blocks_ptr); diff --git a/src/Storages/LiveView/LiveViewEventsBlockInputStream.h b/src/Storages/LiveView/LiveViewEventsBlockInputStream.h index 1f86c41a078..ac5e7e3d6fd 100644 --- a/src/Storages/LiveView/LiveViewEventsBlockInputStream.h +++ b/src/Storages/LiveView/LiveViewEventsBlockInputStream.h @@ -25,7 +25,7 @@ namespace DB { /** Implements LIVE VIEW table WATCH EVENTS input stream. - * Keeps stream alive by outputing blocks with no rows + * Keeps stream alive by outputting blocks with no rows * based on period specified by the heartbeat interval. */ class LiveViewEventsBlockInputStream : public IBlockInputStream @@ -161,7 +161,7 @@ protected: if (!active) return { Block(), false }; /// If we are done iterating over our blocks - /// and there are new blocks availble then get them + /// and there are new blocks available then get them if (blocks.get() != (*blocks_ptr).get()) { blocks = (*blocks_ptr); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 6f4585b819d..801d5b09a7c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -87,7 +87,7 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( disk->createDirectories(part_path); } -// Implemetation is splitted into static functions for ability +// Implementation is split into static functions for ability /// of making unit tests without creation instance of IMergeTreeDataPartWriter, /// which requires a lot of dependencies and access to filesystem. static size_t computeIndexGranularityImpl( diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 50163d892e2..99629144680 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -642,7 +642,7 @@ bool SplitTokenExtractor::next(const char * data, size_t len, size_t * pos, size while (*pos < len) { -#if defined(__SSE2__) && !defined(MEMORY_SANITIZER) /// We read uninitialized bytes and decide on the calcualted mask +#if defined(__SSE2__) && !defined(MEMORY_SANITIZER) /// We read uninitialized bytes and decide on the calculated mask // NOTE: we assume that `data` string is padded from the right with 15 bytes. const __m128i haystack = _mm_loadu_si128(reinterpret_cast(data + *pos)); const size_t haystack_length = 16; @@ -663,7 +663,7 @@ bool SplitTokenExtractor::next(const char * data, size_t len, size_t * pos, size const auto alpha_upper_end = _mm_set1_epi8('Z' + 1); const auto zero = _mm_set1_epi8(0); - // every bit represents if `haystack` character `c` statisfies condition: + // every bit represents if `haystack` character `c` satisfies condition: // (c < 0) || (c > '0' - 1 && c < '9' + 1) || (c > 'a' - 1 && c < 'z' + 1) || (c > 'A' - 1 && c < 'Z' + 1) // < 0 since _mm_cmplt_epi8 threats chars as SIGNED, and so all chars > 0x80 are negative. const int result_bitmask = _mm_movemask_epi8(_mm_or_si128(_mm_or_si128(_mm_or_si128( diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h index 925fdee5107..2b2ed36fc18 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h +++ b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h @@ -43,7 +43,7 @@ private: std::shared_ptr pool; size_t min_marks_to_read; - /// Last part readed in this thread + /// Last part read in this thread std::string last_readed_part_name; /// Names from header. Used in order to order columns in read blocks. Names ordered_names; diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 6bd5c36e757..27baa631a81 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -86,7 +86,7 @@ ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() void ReadBufferFromRabbitMQConsumer::initExchange() { - /* This direct-exchange is used for default implemenation and for INSERT query (so it is always declared). If exchange_type + /* This direct-exchange is used for default implementation and for INSERT query (so it is always declared). If exchange_type * is not set, then there are only two exchanges - external, defined by the client, and local, unique for each table (default). * This strict division to external and local exchanges is needed to avoid too much complexity with defining exchange_name * for INSERT query producer and, in general, it is better to distinguish them into separate ones. diff --git a/src/Storages/tests/gtest_SplitTokenExtractor.cpp b/src/Storages/tests/gtest_SplitTokenExtractor.cpp index d2c1c3ebd76..b5a26c9cd8e 100644 --- a/src/Storages/tests/gtest_SplitTokenExtractor.cpp +++ b/src/Storages/tests/gtest_SplitTokenExtractor.cpp @@ -177,7 +177,7 @@ INSTANTIATE_TEST_SUITE_P(SIMD_Cases, {"abcdef"} }, { - "Token crosses bounday of 16-byte chunk", + "Token crosses boundary of 16-byte chunk", " abcdef", {"abcdef"} }, From 0cbbe153cdbd09463638780e27dbd0d02cc1095f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 04:21:04 +0300 Subject: [PATCH 175/374] Fix typos, the last 1% --- base/common/DateLUTImpl.h | 2 +- base/common/sleep.cpp | 2 +- base/daemon/BaseDaemon.h | 2 +- base/ext/chrono_io.h | 12 ++++++------ base/glibc-compatibility/musl/lgamma.c | 2 +- base/loggers/Loggers.h | 2 +- base/mysqlxx/mysqlxx.h | 4 +--- base/pcg-random/pcg_extras.hpp | 4 ++-- base/pcg-random/pcg_random.hpp | 4 ++-- programs/client/Client.cpp | 2 +- programs/client/QueryFuzzer.h | 2 +- programs/copier/ClusterCopier.cpp | 8 ++++---- programs/copier/ClusterCopierApp.h | 2 +- programs/copier/TaskTableAndShard.h | 10 +++++----- programs/local/LocalServer.cpp | 2 +- programs/server/config.xml | 8 ++++---- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Parsers/ExpressionElementParsers.cpp | 2 +- utils/checksum-for-compressed-block/main.cpp | 2 +- utils/ci/README.md | 2 +- utils/ci/default-config | 2 +- utils/clickhouse-docker | 2 +- utils/github-hook/hook.py | 2 +- utils/github/backport.py | 2 +- utils/github/query.py | 2 +- utils/grammar/ClickHouseLexer.g4 | 4 ++-- utils/junit_to_html/junit-noframes.xsl | 2 +- utils/release/push_packages | 2 +- utils/s3tools/s3uploader | 2 +- utils/test_history/README.md | 12 ++++++------ utils/test_history/test-history | 2 +- 31 files changed, 54 insertions(+), 56 deletions(-) diff --git a/base/common/DateLUTImpl.h b/base/common/DateLUTImpl.h index 7945834be50..97301fb1bee 100644 --- a/base/common/DateLUTImpl.h +++ b/base/common/DateLUTImpl.h @@ -404,7 +404,7 @@ public: a date at start of january) In this case one can get 53 for the first week of next year. This flag ensures that the week is relevant for the given year. Note that this flag is only - releveant if WeekModeFlag::JANUARY is not set. + relevant if WeekModeFlag::JANUARY is not set. If set Week is in range 1-53. diff --git a/base/common/sleep.cpp b/base/common/sleep.cpp index 85bbc8edfcc..4974d65a39d 100644 --- a/base/common/sleep.cpp +++ b/base/common/sleep.cpp @@ -26,7 +26,7 @@ void sleepForNanoseconds(uint64_t nanoseconds) if (timebase_info.denom == 0) mach_timebase_info(&timebase_info); - uint64_t time_to_wait = nanoseconds * timebase_info.denom / timebase_info.numer; + uint64_t time_to_wait = nanoseconds * timebase_info.denom / timebase_info.number; uint64_t now = mach_absolute_time(); while (mach_wait_until(now + time_to_wait) != KERN_SUCCESS); diff --git a/base/daemon/BaseDaemon.h b/base/daemon/BaseDaemon.h index 41d4ad58869..08bbfa291c4 100644 --- a/base/daemon/BaseDaemon.h +++ b/base/daemon/BaseDaemon.h @@ -29,7 +29,7 @@ namespace Poco { class TaskManager; } -/// \brief Base class for applications that can run as deamons. +/// \brief Base class for applications that can run as daemons. /// /// \code /// # Some possible command line options: diff --git a/base/ext/chrono_io.h b/base/ext/chrono_io.h index 0b1c47d3874..967b7c5e475 100644 --- a/base/ext/chrono_io.h +++ b/base/ext/chrono_io.h @@ -26,12 +26,12 @@ namespace ext } template > - std::string to_string(const std::chrono::duration & dur) + std::string to_string(const std::chrono::duration & duration) { - auto seconds_as_int = std::chrono::duration_cast(dur); - if (seconds_as_int == dur) + auto seconds_as_int = std::chrono::duration_cast(duration); + if (seconds_as_int == duration) return std::to_string(seconds_as_int.count()) + "s"; - auto seconds_as_double = std::chrono::duration_cast>(dur); + auto seconds_as_double = std::chrono::duration_cast>(duration); return std::to_string(seconds_as_double.count()) + "s"; } @@ -42,8 +42,8 @@ namespace ext } template > - std::ostream & operator<<(std::ostream & o, const std::chrono::duration & dur) + std::ostream & operator<<(std::ostream & o, const std::chrono::duration & duration) { - return o << to_string(dur); + return o << to_string(duration); } } diff --git a/base/glibc-compatibility/musl/lgamma.c b/base/glibc-compatibility/musl/lgamma.c index b0e4f3aa537..fbbe1fedc92 100644 --- a/base/glibc-compatibility/musl/lgamma.c +++ b/base/glibc-compatibility/musl/lgamma.c @@ -24,7 +24,7 @@ * = log(6.3*5.3) + lgamma(5.3) * = log(6.3*5.3*4.3*3.3*2.3) + lgamma(2.3) * 2. Polynomial approximation of lgamma around its - * minimun ymin=1.461632144968362245 to maintain monotonicity. + * minimum ymin=1.461632144968362245 to maintain monotonicity. * On [ymin-0.23, ymin+0.27] (i.e., [1.23164,1.73163]), use * Let z = x-ymin; * lgamma(x) = -1.214862905358496078218 + z^2*poly(z) diff --git a/base/loggers/Loggers.h b/base/loggers/Loggers.h index 0095516a738..264bf23a1f7 100644 --- a/base/loggers/Loggers.h +++ b/base/loggers/Loggers.h @@ -21,7 +21,7 @@ public: std::optional getLayer() const { - return layer; /// layer setted in inheritor class BaseDaemonApplication. + return layer; /// layer set in inheritor class BaseDaemonApplication. } void setTextLog(std::shared_ptr log, int max_priority); diff --git a/base/mysqlxx/mysqlxx.h b/base/mysqlxx/mysqlxx.h index 179d550519e..d64dadfc367 100644 --- a/base/mysqlxx/mysqlxx.h +++ b/base/mysqlxx/mysqlxx.h @@ -53,8 +53,6 @@ * These assumptions are specific for Yandex.Metrica. Your mileage may vary. * * mysqlxx could not be considered as separate full-featured library, - * because it is developed from the principle - "everything that we don't need is not implemented", - * and also the library depends on some other libraries from Yandex.Metrica code. - * (dependencied could be easily removed if necessary). + * because it is developed from the principle - "everything that we don't need is not implemented". * It is assumed that the user will add all missing functionality that is needed. */ diff --git a/base/pcg-random/pcg_extras.hpp b/base/pcg-random/pcg_extras.hpp index 118b726dd57..b71e859a25f 100644 --- a/base/pcg-random/pcg_extras.hpp +++ b/base/pcg-random/pcg_extras.hpp @@ -110,7 +110,7 @@ namespace pcg_extras { /* * C++ requires us to be able to serialize RNG state by printing or reading * it from a stream. Because we use 128-bit ints, we also need to be able - * ot print them, so here is code to do so. + * or print them, so here is code to do so. * * This code provides enough functionality to print 128-bit ints in decimal * and zero-padded in hex. It's not a full-featured implementation. @@ -253,7 +253,7 @@ inline std::istream& operator>>(std::istream& in, uint8_t& value) */ /* - * XorShifts are invertable, but they are someting of a pain to invert. + * XorShifts are invertable, but they are something of a pain to invert. * This function backs them out. It's used by the whacky "inside out" * generator defined later. */ diff --git a/base/pcg-random/pcg_random.hpp b/base/pcg-random/pcg_random.hpp index ea441013d18..d96d5895b31 100644 --- a/base/pcg-random/pcg_random.hpp +++ b/base/pcg-random/pcg_random.hpp @@ -174,7 +174,7 @@ PCG_DEFINE_CONSTANT(pcg128_t, default, increment, * period * specific stream - the constant can be changed at any time, selecting * a different random sequence - * unique stream - the constant is based on the memory addresss of the + * unique stream - the constant is based on the memory address of the * object, thus every RNG has its own unique sequence * * This variation is provided though mixin classes which define a function @@ -352,7 +352,7 @@ protected: * (reducing register pressure). * * Given the high level of parameterization, the code has to use some - * template-metaprogramming tricks to handle some of the suble variations + * template-metaprogramming tricks to handle some of the subtle variations * involved. */ diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 440d080637f..0ffe1552f15 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -789,7 +789,7 @@ private: // in particular, it can't distinguish the end of partial input buffer // and the final end of input file. This means we have to try to split // the input into separate queries here. Two patterns of input are - // especially interesing: + // especially interesting: // 1) multiline query: // select 1 // from system.numbers; diff --git a/programs/client/QueryFuzzer.h b/programs/client/QueryFuzzer.h index db1102a94d8..0c7cec8dc84 100644 --- a/programs/client/QueryFuzzer.h +++ b/programs/client/QueryFuzzer.h @@ -40,7 +40,7 @@ struct QueryFuzzer // ASTPtr to point to new AST with some random changes. void fuzzMain(ASTPtr & ast); - // Variuos helper functions follow, normally you shouldn't have to call them. + // Various helper functions follow, normally you shouldn't have to call them. Field getRandomField(int type); Field fuzzField(Field field); ASTPtr getRandomColumnLike(); diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index d5544703aa2..b3d1ca7bcec 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -328,7 +328,7 @@ void ClusterCopier::process(const ConnectionTimeouts & timeouts) /* * Creates task worker node and checks maximum number of workers not to exceed the limit. - * To achive this we have to check version of workers_version_path node and create current_worker_path + * To achieve this we have to check version of workers_version_path node and create current_worker_path * node atomically. * */ @@ -529,7 +529,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t inject_fault = value < move_fault_probability; } - LOG_DEBUG(log, "Try to move {} to destionation table", partition_name); + LOG_DEBUG(log, "Try to move {} to destination table", partition_name); auto zookeeper = context.getZooKeeper(); @@ -1001,7 +1001,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab } catch (...) { - tryLogCurrentException(log, "Some error occured while moving pieces to destination table for partition " + partition_name); + tryLogCurrentException(log, "Some error occurred while moving pieces to destination table for partition " + partition_name); } } } @@ -1649,7 +1649,7 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout dropAndCreateLocalTable(create_table_split_piece_ast); - /// Create auxilary split tables for each piece + /// Create auxiliary split tables for each piece for (const auto & piece_number : ext::range(0, task_table.number_of_splits)) { const auto & storage_piece_split_ast = task_table.auxiliary_engine_split_asts[piece_number]; diff --git a/programs/copier/ClusterCopierApp.h b/programs/copier/ClusterCopierApp.h index 173aacc4361..257b10cf196 100644 --- a/programs/copier/ClusterCopierApp.h +++ b/programs/copier/ClusterCopierApp.h @@ -13,7 +13,7 @@ * Implementation details: * * cluster-copier workers pull each partition of each shard of the source cluster and push it to the destination cluster through - * Distributed table (to preform data resharding). So, worker job is a partition of a source shard. + * Distributed table (to perform data resharding). So, worker job is a partition of a source shard. * A job has three states: Active, Finished and Abandoned. Abandoned means that worker died and did not finish the job. * * If an error occurred during the copying (a worker failed or a worker did not finish the INSERT), then the whole partition (on diff --git a/programs/copier/TaskTableAndShard.h b/programs/copier/TaskTableAndShard.h index 0ac533d9209..11ceffd12cd 100644 --- a/programs/copier/TaskTableAndShard.h +++ b/programs/copier/TaskTableAndShard.h @@ -50,7 +50,7 @@ struct TaskTable bool isReplicatedTable() const { return engine_push_zk_path != ""; } - /// Partitions will be splitted into number-of-splits pieces. + /// Partitions will be split into number-of-splits pieces. /// Each piece will be copied independently. (10 by default) size_t number_of_splits; @@ -91,8 +91,8 @@ struct TaskTable ASTPtr main_engine_split_ast; /* - * To copy partiton piece form one cluster to another we have to use Distributed table. - * In case of usage separate table (engine_push) for each partiton piece, + * To copy partition piece form one cluster to another we have to use Distributed table. + * In case of usage separate table (engine_push) for each partition piece, * we have to use many Distributed tables. * */ ASTs auxiliary_engine_split_asts; @@ -113,7 +113,7 @@ struct TaskTable /** * Prioritized list of shards * all_shards contains information about all shards in the table. - * So we have to check whether particular shard have current partiton or not while processing. + * So we have to check whether particular shard have current partition or not while processing. */ TasksShard all_shards; TasksShard local_shards; @@ -122,7 +122,7 @@ struct TaskTable ClusterPartitions cluster_partitions; NameSet finished_cluster_partitions; - /// Parition names to process in user-specified order + /// Partition names to process in user-specified order Strings ordered_partition_names; ClusterPartition & getClusterPartition(const String & partition_name) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index e3a5306a6f1..ca348382281 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -114,7 +114,7 @@ void LocalServer::tryInitPath() if (path.empty()) { throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Cannot work with emtpy storage path that is explicitly specified" + "Cannot work with empty storage path that is explicitly specified" " by the --path option. Please check the program options and" " correct the --path."); } diff --git a/programs/server/config.xml b/programs/server/config.xml index e494a539a98..506e7d0b006 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1,7 +1,7 @@ @@ -103,7 +103,7 @@ 9009 - + @@ -455,7 +455,7 @@ --> - + + N/A diff --git a/utils/release/push_packages b/utils/release/push_packages index 68d72bb39fe..43e75a723da 100755 --- a/utils/release/push_packages +++ b/utils/release/push_packages @@ -188,7 +188,7 @@ def _fix_args(args): if __name__ == "__main__": logging.basicConfig(level=logging.INFO, format='%(asctime)s %(message)s') - parser = argparse.ArgumentParser(description="Programm to push clickhouse packages to repository") + parser = argparse.ArgumentParser(description="Program to push clickhouse packages to repository") parser.add_argument('--deb-directory') parser.add_argument('--rpm-directory') parser.add_argument('--tgz-directory') diff --git a/utils/s3tools/s3uploader b/utils/s3tools/s3uploader index 66322be623f..91fb60ed962 100755 --- a/utils/s3tools/s3uploader +++ b/utils/s3tools/s3uploader @@ -90,7 +90,7 @@ if __name__ == "__main__": help='Path to clickhouse database on filesystem') parser.add_argument('--s3-path', help='Path in s3, where to upload file') parser.add_argument('--tmp-prefix', default='/tmp', - help='Prefix to store temporay downloaded file') + help='Prefix to store temporary downloaded file') data_group = parser.add_mutually_exclusive_group(required=True) table_name_argument = data_group.add_argument('--table-name', help='Name of table with database, if you are uploading partitions') diff --git a/utils/test_history/README.md b/utils/test_history/README.md index 1de9bf0a4ab..2d4af866012 100644 --- a/utils/test_history/README.md +++ b/utils/test_history/README.md @@ -10,11 +10,11 @@ $ test-history --token XXX --since '2020-01-22 00:00:00' --substr Performance' +---------------------|---------|--------------------+ | Date | SHA | Performance test | +=====================+=========+====================+ -| 2020-01-22 12:54:59 | 47ffa40 | succes | +| 2020-01-22 12:54:59 | 47ffa40 | success | +---------------------|---------|--------------------+ | 2020-01-22 13:06:16 | 0d484be | failure | +---------------------|---------|--------------------+ -| 2020-01-22 14:18:34 | 289f169 | succes | +| 2020-01-22 14:18:34 | 289f169 | success | +---------------------|---------|--------------------+ | 2020-01-22 14:27:27 | e357c6f | not run | +---------------------|---------|--------------------+ @@ -26,11 +26,11 @@ $ test-history --token XXX --since '2020-01-22 00:00:00' --substr Performance' +---------------------|---------|--------------------+ | 2020-01-22 16:58:36 | d68f8d1 | pending | +---------------------|---------|--------------------+ -| 2020-01-22 17:59:43 | ba7ab32 | succes | +| 2020-01-22 17:59:43 | ba7ab32 | success | +---------------------|---------|--------------------+ | 2020-01-22 18:32:38 | eadb902 | failure | +---------------------|---------|--------------------+ -| 2020-01-22 19:11:34 | 8f241ea | succes | +| 2020-01-22 19:11:34 | 8f241ea | success | +---------------------|---------|--------------------+ | 2020-01-22 19:56:49 | f0b7422 | failure | +---------------------|---------|--------------------+ @@ -40,11 +40,11 @@ $ test-history --token XXX --since '2020-01-22 00:00:00' --substr Performance' +---------------------|---------|--------------------+ | 2020-01-22 23:09:23 | 8cfe9a4 | failure | +---------------------|---------|--------------------+ -| 2020-01-23 00:10:33 | a02b59f | succes | +| 2020-01-23 00:10:33 | a02b59f | success | +---------------------|---------|--------------------+ | 2020-01-23 05:56:11 | 48b3f33 | failure | +---------------------|---------|--------------------+ -| 2020-01-23 05:56:54 | d807088 | succes | +| 2020-01-23 05:56:54 | d807088 | success | +---------------------|---------|--------------------+ | 2020-01-23 06:01:48 | 2e84949 | failure | +---------------------|---------|--------------------+ diff --git a/utils/test_history/test-history b/utils/test_history/test-history index 783f25ff822..dca62625c9f 100755 --- a/utils/test_history/test-history +++ b/utils/test_history/test-history @@ -11,7 +11,7 @@ from termcolor import colored import sys COLORMAP = { - "success": colored("succes", 'green'), + "success": colored("success", 'green'), "failure": colored("failure", 'red'), "error": colored("error", 'red'), "pending": colored("pending", 'yellow'), From 6be6ac9c142bf1710fe3f2b8c0734b0f12a2dfa3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 04:58:04 +0300 Subject: [PATCH 176/374] Possibly fix clang-tidy --- src/Columns/ColumnString.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 5e0890e11e6..948eecf24f0 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -23,7 +23,8 @@ namespace ErrorCodes ColumnString::ColumnString(const ColumnString & src) - : offsets(src.offsets.begin(), src.offsets.end()), + : COWHelper(src), + offsets(src.offsets.begin(), src.offsets.end()), chars(src.chars.begin(), src.chars.end()) { if (!offsets.empty()) From add08dbc47594c661ec5c79c957ef55b4814145f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 04:58:40 +0300 Subject: [PATCH 177/374] Fix style --- src/Columns/ColumnString.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 948eecf24f0..6c84107caae 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -38,7 +38,6 @@ ColumnString::ColumnString(const ColumnString & src) } - MutableColumnPtr ColumnString::cloneResized(size_t to_size) const { auto res = ColumnString::create(); From 6663f21fdd7c07b2ab549732d2f406133d839803 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 05:02:36 +0300 Subject: [PATCH 178/374] Fix error --- src/Functions/lcm.cpp | 5 +++-- tests/queries/0_stateless/01435_lcm_overflow.reference | 8 ++++---- tests/queries/0_stateless/01435_lcm_overflow.sql | 8 ++++---- 3 files changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Functions/lcm.cpp b/src/Functions/lcm.cpp index 98399b8374b..e014fa64cc3 100644 --- a/src/Functions/lcm.cpp +++ b/src/Functions/lcm.cpp @@ -49,9 +49,10 @@ struct LCMImpl */ using Int = typename NumberTraits::ToInteger::Type; + using Unsigned = std::make_unsigned_t; - Int val1 = abs(a) / std::gcd(Int(a), Int(b)); - Int val2 = abs(b); + Unsigned val1 = abs(a) / std::gcd(Int(a), Int(b)); + Unsigned val2 = abs(b); /// Overflow in implementation specific way. return Result(val1 * val2); diff --git a/tests/queries/0_stateless/01435_lcm_overflow.reference b/tests/queries/0_stateless/01435_lcm_overflow.reference index 56401b21045..eebd14705df 100644 --- a/tests/queries/0_stateless/01435_lcm_overflow.reference +++ b/tests/queries/0_stateless/01435_lcm_overflow.reference @@ -2,7 +2,7 @@ 30 30 30 -18446744073709551360 --256 --256 --256 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/01435_lcm_overflow.sql b/tests/queries/0_stateless/01435_lcm_overflow.sql index 3285514ac1d..f70200eb2d8 100644 --- a/tests/queries/0_stateless/01435_lcm_overflow.sql +++ b/tests/queries/0_stateless/01435_lcm_overflow.sql @@ -4,7 +4,7 @@ SELECT lcm(15, -10); SELECT lcm(-15, -10); -- Implementation specific result on overflow: -SELECT lcm(256, 9223372036854775807); -SELECT lcm(256, -9223372036854775807); -SELECT lcm(-256, 9223372036854775807); -SELECT lcm(-256, -9223372036854775807); +SELECT ignore(lcm(256, 9223372036854775807)); +SELECT ignore(lcm(256, -9223372036854775807)); +SELECT ignore(lcm(-256, 9223372036854775807)); +SELECT ignore(lcm(-256, -9223372036854775807)); From 11228a127dfad4438d20d0161c499e86235ac675 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 05:08:36 +0300 Subject: [PATCH 179/374] Check for copy-pasted GPL code --- utils/check-style/check-style | 3 +++ 1 file changed, 3 insertions(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index e3e90e17c7a..939eb3ecca8 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -65,3 +65,6 @@ find $ROOT_PATH/tests/queries -iname '*fail*' | grep . && echo 'Tests should not # All the submodules should be from https://github.com/ find $ROOT_PATH -name '.gitmodules' | while read i; do grep -F 'url = ' $i | grep -v -F 'https://github.com/' && echo 'All the submodules should be from https://github.com/'; done + +# There shouldn't be any code snippets under GPL or LGPL +find $ROOT_PATH/{src,base,programs} -name '*.h' -or -name '*.cpp' 2>/dev/null | xargs grep -i -F 'General Public License' && echo "There shouldn't be any code snippets under GPL or LGPL" From f6da5d58d061b828c77742f5b10f8971ce624a49 Mon Sep 17 00:00:00 2001 From: Konstantin Podshumok Date: Sat, 8 Aug 2020 05:29:55 +0300 Subject: [PATCH 180/374] Don't enable march=native on arm by default --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 9d1f46833b5..6e82e18f823 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -176,7 +176,7 @@ if(NOT DISABLE_CPU_OPTIMIZE) include(cmake/cpu_features.cmake) endif() -option(ARCH_NATIVE "Enable -march=native compiler flag" ${ARCH_ARM}) +option(ARCH_NATIVE "Enable -march=native compiler flag" 0) if (ARCH_NATIVE) set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native") endif () From 0f79eb3cc518977a46b8020c6e317cae15ff27b5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 06:42:42 +0300 Subject: [PATCH 181/374] Embed configs into binary --- CMakeLists.txt | 7 +++++ base/common/CMakeLists.txt | 1 + base/common/DateLUTImpl.cpp | 20 +++----------- base/common/getResource.cpp | 24 ++++++++++++++++ base/common/getResource.h | 7 +++++ contrib/cctz-cmake/CMakeLists.txt | 7 ----- programs/server/CMakeLists.txt | 37 +++++++++++++++++++------ programs/server/config.xml | 2 +- programs/server/embedded.xml | 40 +++++++++++++++++++++++++++ src/Common/Config/ConfigProcessor.cpp | 28 ++++++++++++++++++- 10 files changed, 140 insertions(+), 33 deletions(-) create mode 100644 base/common/getResource.cpp create mode 100644 base/common/getResource.h create mode 100644 programs/server/embedded.xml diff --git a/CMakeLists.txt b/CMakeLists.txt index e137c2267f0..f7f7cc65ab6 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -142,6 +142,13 @@ endif () # Make sure the final executable has symbols exported set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic") +find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy") +if (OBJCOPY_PATH) + message(STATUS "Using objcopy: ${OBJCOPY_PATH}.") +else () + message(FATAL_ERROR "Cannot find objcopy.") +endif () + option (ADD_GDB_INDEX_FOR_GOLD "Set to add .gdb-index to resulting binaries for gold linker. NOOP if lld is used." 0) if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") if (LINKER_NAME STREQUAL "lld") diff --git a/base/common/CMakeLists.txt b/base/common/CMakeLists.txt index f09335f0ca0..903168a0dd4 100644 --- a/base/common/CMakeLists.txt +++ b/base/common/CMakeLists.txt @@ -17,6 +17,7 @@ set (SRCS sleep.cpp terminalColors.cpp errnoToString.cpp + getResource.cpp ) if (ENABLE_REPLXX) diff --git a/base/common/DateLUTImpl.cpp b/base/common/DateLUTImpl.cpp index 83a4d1e9149..50620e21b8f 100644 --- a/base/common/DateLUTImpl.cpp +++ b/base/common/DateLUTImpl.cpp @@ -3,11 +3,9 @@ #include #include #include -#include +#include #include -#include - #include #include #include @@ -213,19 +211,9 @@ namespace cctz_extension const std::string & name, const std::function(const std::string & name)> & fallback) { - std::string name_replaced = name; - std::replace(name_replaced.begin(), name_replaced.end(), '/', '_'); - std::replace(name_replaced.begin(), name_replaced.end(), '-', '_'); - - /// These are the names that are generated by "ld -r -b binary" - std::string symbol_name_data = "_binary_" + name_replaced + "_start"; - std::string symbol_name_size = "_binary_" + name_replaced + "_size"; - - const void * sym_data = dlsym(RTLD_DEFAULT, symbol_name_data.c_str()); - const void * sym_size = dlsym(RTLD_DEFAULT, symbol_name_size.c_str()); - - if (sym_data && sym_size) - return std::make_unique(static_cast(sym_data), unalignedLoad(&sym_size)); + std::string_view resource = getResource(name); + if (!resource.empty()) + return std::make_unique(resource.data(), resource.size()); return fallback(name); } diff --git a/base/common/getResource.cpp b/base/common/getResource.cpp new file mode 100644 index 00000000000..be7f205e7d1 --- /dev/null +++ b/base/common/getResource.cpp @@ -0,0 +1,24 @@ +#include "getResource.h" +#include "unaligned.h" +#include +#include + + +std::string_view getResource(std::string_view name) +{ + std::string name_replaced(name); + std::replace(name_replaced.begin(), name_replaced.end(), '/', '_'); + std::replace(name_replaced.begin(), name_replaced.end(), '-', '_'); + std::replace(name_replaced.begin(), name_replaced.end(), '.', '_'); + + /// These are the names that are generated by "ld -r -b binary" + std::string symbol_name_data = "_binary_" + name_replaced + "_start"; + std::string symbol_name_size = "_binary_" + name_replaced + "_size"; + + const void * sym_data = dlsym(RTLD_DEFAULT, symbol_name_data.c_str()); + const void * sym_size = dlsym(RTLD_DEFAULT, symbol_name_size.c_str()); + + if (sym_data && sym_size) + return { static_cast(sym_data), unalignedLoad(&sym_size) }; + return {}; +} diff --git a/base/common/getResource.h b/base/common/getResource.h new file mode 100644 index 00000000000..8975cc7841e --- /dev/null +++ b/base/common/getResource.h @@ -0,0 +1,7 @@ +#pragma once + +#include + +/// Get resource from binary if exists. Otherwise return empty string view. +/// Resources are data that is embedded into executable at link time. +std::string_view getResource(std::string_view name); diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 0837a366f20..7eb768b8de9 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -28,13 +28,6 @@ if (USE_INTERNAL_CCTZ) if (OS_LINUX AND ARCH_AMD64) - find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy") - if (OBJCOPY_PATH) - message(STATUS "Using objcopy: ${OBJCOPY_PATH}.") - else () - message(FATAL_ERROR "Cannot find objcopy.") - endif () - set (TIMEZONES Africa/Abidjan Africa/Accra diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index 1563f5ac51e..81fb7b6d623 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -16,18 +16,39 @@ set (CLICKHOUSE_SERVER_LINK clickhouse_table_functions string_utils + INTERFACE "-Wl,--whole-archive $ -Wl,--no-whole-archive" + PUBLIC daemon ) clickhouse_program_add(server) -if (GLIBC_COMPATIBILITY) - set (GLIBC_MAX_REQUIRED 2.4 CACHE INTERNAL "") - # temporary disabled. to enable - change 'exit 0' to 'exit $a' - add_test(NAME GLIBC_required_version COMMAND bash -c "readelf -s ${CMAKE_CURRENT_BINARY_DIR}/../clickhouse-server | perl -nE 'END {exit 0 if $a} ++$a, print if /\\x40GLIBC_(\\S+)/ and pack(q{C*}, split /\\./, \$1) gt pack q{C*}, split /\\./, q{${GLIBC_MAX_REQUIRED}}'") - - #add_test(NAME GLIBC_required_version COMMAND bash -c "readelf -s ${CMAKE_CURRENT_BINARY_DIR}/../clickhouse-server | grep '@GLIBC' | grep -oP 'GLIBC_[\\d\\.]+' | sort | uniq | sort --version-sort --reverse | perl -lnE 'warn($_), exit 1 if $_ gt q{GLIBC_${GLIBC_MAX_REQUIRED}}'") # old -endif () - install(FILES config.xml users.xml DESTINATION ${CLICKHOUSE_ETC_DIR}/clickhouse-server COMPONENT clickhouse) + +# Embed default config files as a resource into the binary. +# This is needed for two purposes: +# 1. Allow to run the binary without download of any other files. +# 2. Allow to implement "sudo clickhouse install" tool. + +foreach(CONFIG_FILE config users embedded) + set(CONFIG_OBJ ${CONFIG_FILE}.o) + set(CONFIG_OBJS ${CONFIG_OBJS} ${CONFIG_OBJ}) + + # https://stackoverflow.com/questions/14776463/compile-and-add-an-object-file-from-a-binary-with-cmake + add_custom_command(OUTPUT ${CONFIG_OBJ} + COMMAND cd ${CMAKE_CURRENT_SOURCE_DIR} && ${OBJCOPY_PATH} -I binary -O elf64-x86-64 -B i386 ${CONFIG_FILE}.xml ${CMAKE_CURRENT_BINARY_DIR}/${CONFIG_OBJ} + COMMAND ${OBJCOPY_PATH} --rename-section .data=.rodata,alloc,load,readonly,data,contents + ${CMAKE_CURRENT_BINARY_DIR}/${CONFIG_OBJ} ${CMAKE_CURRENT_BINARY_DIR}/${CONFIG_OBJ}) + + set_source_files_properties(${CONFIG_OBJ} PROPERTIES EXTERNAL_OBJECT true GENERATED true) +endforeach(CONFIG_FILE) + +add_library(clickhouse_server_configs STATIC ${CONFIG_OBJS}) +set_target_properties(clickhouse_server_configs PROPERTIES LINKER_LANGUAGE C) + +# whole-archive prevents symbols from being discarded for unknown reason +# CMake can shuffle each of target_link_libraries arguments with other +# libraries in linker command. To avoid this we hardcode whole-archive +# library into single string. +add_dependencies(clickhouse-server-lib clickhouse_server_configs) diff --git a/programs/server/config.xml b/programs/server/config.xml index e494a539a98..94576197667 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -154,7 +154,7 @@ of the time, in which case a higher number of threads might be required. --> - 10000 + 10000 diff --git a/programs/server/embedded.xml b/programs/server/embedded.xml new file mode 100644 index 00000000000..1ac568682f9 --- /dev/null +++ b/programs/server/embedded.xml @@ -0,0 +1,40 @@ + + + + + trace + true + + + 8123 + 9000 + 9004 + + ./ + + 8589934592 + 5368709120 + true + + + + + + + ::/0 + + + default + default + 1 + + + + + + + + + + + diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 0e36c6ee660..ab25ec35672 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -14,6 +15,8 @@ #include #include #include +#include +#include #define PREPROCESSED_SUFFIX "-preprocessed" @@ -23,6 +26,11 @@ using namespace Poco::XML; namespace DB { +namespace ErrorCodes +{ + extern const int FILE_DOESNT_EXIST; +} + /// For cutting preprocessed path to this base static std::string main_config_path; @@ -440,9 +448,27 @@ XMLDocumentPtr ConfigProcessor::processConfig( zkutil::ZooKeeperNodeCache * zk_node_cache, const zkutil::EventPtr & zk_changed_event) { + XMLDocumentPtr config; LOG_DEBUG(log, "Processing configuration file '{}'.", path); - XMLDocumentPtr config = dom_parser.parse(path); + if (std::filesystem::exists(path)) + { + config = dom_parser.parse(path); + } + else + { + /// When we can use config embedded in binary. + if (path == "config.xml") + { + auto resource = getResource("embedded.xml"); + if (resource.empty()) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Configuration file {} doesn't exist and there is no embedded config", path); + LOG_DEBUG(log, "There is no file '{}', will use embedded config.", path); + config = dom_parser.parseMemory(resource.data(), resource.size()); + } + else + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Configuration file {} doesn't exist", path); + } std::vector contributing_files; contributing_files.push_back(path); From 0d70393bac2aa4668d2f819cda4717b672fdd2c9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 07:55:33 +0300 Subject: [PATCH 182/374] Fix build --- base/common/sleep.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/common/sleep.cpp b/base/common/sleep.cpp index 4974d65a39d..85bbc8edfcc 100644 --- a/base/common/sleep.cpp +++ b/base/common/sleep.cpp @@ -26,7 +26,7 @@ void sleepForNanoseconds(uint64_t nanoseconds) if (timebase_info.denom == 0) mach_timebase_info(&timebase_info); - uint64_t time_to_wait = nanoseconds * timebase_info.denom / timebase_info.number; + uint64_t time_to_wait = nanoseconds * timebase_info.denom / timebase_info.numer; uint64_t now = mach_absolute_time(); while (mach_wait_until(now + time_to_wait) != KERN_SUCCESS); From d9de96716a4b22dd79d6780e4b36e527e8d46ced Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 8 Aug 2020 14:30:50 +0800 Subject: [PATCH 183/374] string to date is not monotonic --- src/Functions/FunctionsConversion.h | 18 ++++++++++++++++-- .../01440_to_date_monotonicity.reference | 1 + 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 7b1b8388866..88dd3621ecf 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1518,13 +1518,27 @@ struct ToDateMonotonicity else if ( (which.isUInt() && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF))) || (which.isInt() && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF))) - || (which.isFloat() && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF)))) + || (which.isFloat() && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF))) + || !type.isValueRepresentedByNumber()) return {}; else return {true, true, true}; } }; +struct ToDateTimeMonotonicity +{ + static bool has() { return true; } + + static IFunction::Monotonicity get(const IDataType & type, const Field &, const Field &) + { + if (type.isValueRepresentedByNumber()) + return {true, true, true}; + else + return {}; + } +}; + /** The monotonicity for the `toString` function is mainly determined for test purposes. * It is doubtful that anyone is looking to optimize queries with conditions `toString(CounterID) = 34`. */ @@ -1592,7 +1606,7 @@ using FunctionToInt64 = FunctionConvert>; using FunctionToFloat64 = FunctionConvert>; using FunctionToDate = FunctionConvert; -using FunctionToDateTime = FunctionConvert; +using FunctionToDateTime = FunctionConvert; using FunctionToDateTime64 = FunctionConvert; using FunctionToUUID = FunctionConvert>; using FunctionToString = FunctionConvert; diff --git a/tests/queries/0_stateless/01440_to_date_monotonicity.reference b/tests/queries/0_stateless/01440_to_date_monotonicity.reference index 529601fb398..30b841d4487 100644 --- a/tests/queries/0_stateless/01440_to_date_monotonicity.reference +++ b/tests/queries/0_stateless/01440_to_date_monotonicity.reference @@ -1,2 +1,3 @@ 0 1970-01-01 2106-02-07 1970-04-11 1970-01-01 2106-02-07 +1970-01-01 03:00:00 2106-02-07 09:28:15 1970-01-01 03:16:40 From 84a8ad7da9ad25f90be30f238d521a3f20917ac1 Mon Sep 17 00:00:00 2001 From: vladimir golovchenko Date: Sat, 8 Aug 2020 01:16:56 -0700 Subject: [PATCH 184/374] Fixed docs: added the missed types of interval (WEEK, MONTH, QUARTER, YEAR) to CREATE/ALTER QUOTA statements. --- docs/en/sql-reference/statements/alter/quota.md | 2 +- docs/en/sql-reference/statements/create/quota.md | 2 +- docs/es/sql-reference/statements/alter.md | 2 +- docs/es/sql-reference/statements/create.md | 2 +- docs/fa/sql-reference/statements/alter.md | 2 +- docs/fa/sql-reference/statements/create.md | 2 +- docs/fr/sql-reference/statements/alter.md | 2 +- docs/fr/sql-reference/statements/create.md | 2 +- docs/ja/sql-reference/statements/alter.md | 2 +- docs/ja/sql-reference/statements/create.md | 2 +- docs/ru/sql-reference/statements/alter.md | 2 +- docs/ru/sql-reference/statements/create/quota.md | 2 +- docs/tr/sql-reference/statements/alter.md | 2 +- docs/tr/sql-reference/statements/create.md | 2 +- docs/zh/sql-reference/statements/alter.md | 2 +- 15 files changed, 15 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/quota.md b/docs/en/sql-reference/statements/alter/quota.md index 0dddf7dea1f..2923fd04c4b 100644 --- a/docs/en/sql-reference/statements/alter/quota.md +++ b/docs/en/sql-reference/statements/alter/quota.md @@ -13,7 +13,7 @@ Syntax: ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name] [RENAME TO new_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/docs/en/sql-reference/statements/create/quota.md b/docs/en/sql-reference/statements/create/quota.md index 209c421a631..7919dc0f6db 100644 --- a/docs/en/sql-reference/statements/create/quota.md +++ b/docs/en/sql-reference/statements/create/quota.md @@ -12,7 +12,7 @@ Syntax: ``` sql CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/docs/es/sql-reference/statements/alter.md b/docs/es/sql-reference/statements/alter.md index 8bda03f71bd..7ee45c89fff 100644 --- a/docs/es/sql-reference/statements/alter.md +++ b/docs/es/sql-reference/statements/alter.md @@ -581,7 +581,7 @@ Cambia las cuotas. ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name] [RENAME TO new_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/docs/es/sql-reference/statements/create.md b/docs/es/sql-reference/statements/create.md index aaefa15807b..b851435286e 100644 --- a/docs/es/sql-reference/statements/create.md +++ b/docs/es/sql-reference/statements/create.md @@ -466,7 +466,7 @@ Crea un [cuota](../../operations/access-rights.md#quotas-management) que se pued ``` sql CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/docs/fa/sql-reference/statements/alter.md b/docs/fa/sql-reference/statements/alter.md index 6afc311f74e..36711201bba 100644 --- a/docs/fa/sql-reference/statements/alter.md +++ b/docs/fa/sql-reference/statements/alter.md @@ -581,7 +581,7 @@ ALTER [ROW] POLICY [IF EXISTS] name [ON CLUSTER cluster_name] ON [database.]tabl ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name] [RENAME TO new_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/docs/fa/sql-reference/statements/create.md b/docs/fa/sql-reference/statements/create.md index 2d826507aa6..c4b7ede05dd 100644 --- a/docs/fa/sql-reference/statements/create.md +++ b/docs/fa/sql-reference/statements/create.md @@ -466,7 +466,7 @@ CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] policy_name [ON CLUSTER cluster ``` sql CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/docs/fr/sql-reference/statements/alter.md b/docs/fr/sql-reference/statements/alter.md index c0206b50a9b..298df06304a 100644 --- a/docs/fr/sql-reference/statements/alter.md +++ b/docs/fr/sql-reference/statements/alter.md @@ -581,7 +581,7 @@ Les changements de quotas. ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name] [RENAME TO new_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/docs/fr/sql-reference/statements/create.md b/docs/fr/sql-reference/statements/create.md index f178b409886..f7b3790baf2 100644 --- a/docs/fr/sql-reference/statements/create.md +++ b/docs/fr/sql-reference/statements/create.md @@ -466,7 +466,7 @@ Crée un [quota](../../operations/access-rights.md#quotas-management) qui peut ``` sql CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/docs/ja/sql-reference/statements/alter.md b/docs/ja/sql-reference/statements/alter.md index c5f3655411a..057cbebae7f 100644 --- a/docs/ja/sql-reference/statements/alter.md +++ b/docs/ja/sql-reference/statements/alter.md @@ -581,7 +581,7 @@ ALTER [ROW] POLICY [IF EXISTS] name [ON CLUSTER cluster_name] ON [database.]tabl ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name] [RENAME TO new_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/docs/ja/sql-reference/statements/create.md b/docs/ja/sql-reference/statements/create.md index bc5a5f4e401..ae518dbfac8 100644 --- a/docs/ja/sql-reference/statements/create.md +++ b/docs/ja/sql-reference/statements/create.md @@ -466,7 +466,7 @@ CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] policy_name [ON CLUSTER cluster ``` sql CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/docs/ru/sql-reference/statements/alter.md b/docs/ru/sql-reference/statements/alter.md index 1ad7f270c9d..e923f9c5682 100644 --- a/docs/ru/sql-reference/statements/alter.md +++ b/docs/ru/sql-reference/statements/alter.md @@ -589,7 +589,7 @@ ALTER [ROW] POLICY [IF EXISTS] name [ON CLUSTER cluster_name] ON [database.]tabl ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name] [RENAME TO new_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/docs/ru/sql-reference/statements/create/quota.md b/docs/ru/sql-reference/statements/create/quota.md index e351259632f..6351de2d38a 100644 --- a/docs/ru/sql-reference/statements/create/quota.md +++ b/docs/ru/sql-reference/statements/create/quota.md @@ -12,7 +12,7 @@ toc_title: Квота ``` sql CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/docs/tr/sql-reference/statements/alter.md b/docs/tr/sql-reference/statements/alter.md index 6fad1b12541..6563dca1717 100644 --- a/docs/tr/sql-reference/statements/alter.md +++ b/docs/tr/sql-reference/statements/alter.md @@ -581,7 +581,7 @@ Kotaları değiştirir. ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name] [RENAME TO new_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/docs/tr/sql-reference/statements/create.md b/docs/tr/sql-reference/statements/create.md index 1bb648ed580..79bdb45f9e4 100644 --- a/docs/tr/sql-reference/statements/create.md +++ b/docs/tr/sql-reference/statements/create.md @@ -466,7 +466,7 @@ Oluşturur bir [kota](../../operations/access-rights.md#quotas-management) bu bi ``` sql CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/docs/zh/sql-reference/statements/alter.md b/docs/zh/sql-reference/statements/alter.md index 676fd65831d..d9e9bebf339 100644 --- a/docs/zh/sql-reference/statements/alter.md +++ b/docs/zh/sql-reference/statements/alter.md @@ -568,7 +568,7 @@ ALTER [ROW] POLICY [IF EXISTS] name [ON CLUSTER cluster_name] ON [database.]tabl ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name] [RENAME TO new_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] From 679c2c0a7a0fe49ef54a02b7a2ab27e49625220e Mon Sep 17 00:00:00 2001 From: vladimir golovchenko Date: Sat, 8 Aug 2020 01:19:48 -0700 Subject: [PATCH 185/374] Fixed documentation blocks: added the missed types of interval (week, month, quarter, year) to CREATE/ALTER QUOTA statements. --- src/Parsers/ASTCreateQuotaQuery.h | 4 ++-- src/Parsers/ParserCreateQuotaQuery.h | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ASTCreateQuotaQuery.h b/src/Parsers/ASTCreateQuotaQuery.h index 1671ae1d00f..a1269afafa6 100644 --- a/src/Parsers/ASTCreateQuotaQuery.h +++ b/src/Parsers/ASTCreateQuotaQuery.h @@ -12,7 +12,7 @@ class ASTRolesOrUsersSet; /** CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name * [KEYED BY {none | user_name | ip_address | client_key | client_key, user_name | client_key, ip_address} | NOT KEYED] - * [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day} + * [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day | week | month | quarter | year} * {MAX {{queries | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number} [,...] | * NO LIMITS | TRACKING ONLY} [,...]] * [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] @@ -20,7 +20,7 @@ class ASTRolesOrUsersSet; * ALTER QUOTA [IF EXISTS] name * [RENAME TO new_name] * [KEYED BY {none | user_name | ip_address | client_key | client_key, user_name | client_key, ip_address} | NOT KEYED] - * [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day} + * [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day | week | month | quarter | year} * {MAX {{queries | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number} [,...] | * NO LIMITS | TRACKING ONLY} [,...]] * [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] diff --git a/src/Parsers/ParserCreateQuotaQuery.h b/src/Parsers/ParserCreateQuotaQuery.h index 03021985357..1f1df0588bd 100644 --- a/src/Parsers/ParserCreateQuotaQuery.h +++ b/src/Parsers/ParserCreateQuotaQuery.h @@ -8,7 +8,7 @@ namespace DB /** Parses queries like * CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name * [KEYED BY {none | user_name | ip_address | client_key | client_key, user_name | client_key, ip_address} | NOT KEYED] - * [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day} + * [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day | week | month | quarter | year} * {MAX {{queries | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number} [,...] | * NO LIMITS | TRACKING ONLY} [,...]] * [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] @@ -16,7 +16,7 @@ namespace DB * ALTER QUOTA [IF EXISTS] name * [RENAME TO new_name] * [KEYED BY {none | user_name | ip_address | client_key | client_key, user_name | client_key, ip_address} | NOT KEYED] - * [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day} + * [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day | week | month | quarter | year} * {MAX {{queries | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number} [,...] | * NO LIMITS | TRACKING ONLY} [,...]] * [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] From a45a85f9e535703cbf97b368fc66086cd237765b Mon Sep 17 00:00:00 2001 From: vladimir golovchenko Date: Sat, 8 Aug 2020 01:22:58 -0700 Subject: [PATCH 186/374] Fixed tests: added the missed types of interval (WEEK, MONTH, QUARTER, YEAR) to CREATE/ALTER QUOTA statements. --- .../rbac/requirements/requirements.md | 18 +++++++++--------- .../rbac/requirements/requirements.py | 18 +++++++++--------- .../testflows/rbac/tests/syntax/alter_quota.py | 4 ++-- .../rbac/tests/syntax/create_quota.py | 4 ++-- 4 files changed, 22 insertions(+), 22 deletions(-) diff --git a/tests/testflows/rbac/requirements/requirements.md b/tests/testflows/rbac/requirements/requirements.md index e773d74201f..b9833afe536 100644 --- a/tests/testflows/rbac/requirements/requirements.md +++ b/tests/testflows/rbac/requirements/requirements.md @@ -1985,9 +1985,9 @@ a period of time over for which the quota SHALL apply using the `FOR INTERVAL` clause in the `CREATE QUOTA` statement. This statement SHALL also support a number and a time period which will be one -of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be: +of `{SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}`. Thus, the complete syntax SHALL be: -`FOR INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some real number +`FOR INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}` where number is some real number to define the interval. @@ -1999,9 +1999,9 @@ a period of time over for which the quota SHALL apply using the `FOR RANDOMIZED INTERVAL` clause in the `CREATE QUOTA` statement. This statement SHALL also support a number and a time period which will be one -of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be: +of `{SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}`. Thus, the complete syntax SHALL be: -`FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some +`FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}` where number is some real number to define the interval. ##### RQ.SRS-006.RBAC.Quota.Create.Queries @@ -2107,7 +2107,7 @@ version: 1.0 ```sql CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] @@ -2146,9 +2146,9 @@ a period of time over for which the quota SHALL apply using the `FOR INTERVAL` clause in the `ALTER QUOTA` statement. This statement SHALL also support a number and a time period which will be one -of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be: +of `{SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}`. Thus, the complete syntax SHALL be: -`FOR INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some real number +`FOR INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}` where number is some real number to define the interval. ##### RQ.SRS-006.RBAC.Quota.Alter.Interval.Randomized @@ -2159,9 +2159,9 @@ a period of time over for which the quota SHALL apply using the `FOR RANDOMIZED INTERVAL` clause in the `ALTER QUOTA` statement. This statement SHALL also support a number and a time period which will be one -of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be: +of `{SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}`. Thus, the complete syntax SHALL be: -`FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some +`FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}` where number is some real number to define the interval. ##### RQ.SRS-006.RBAC.Quota.Alter.Queries diff --git a/tests/testflows/rbac/requirements/requirements.py b/tests/testflows/rbac/requirements/requirements.py index 854ad5bab41..e5dabcc9503 100644 --- a/tests/testflows/rbac/requirements/requirements.py +++ b/tests/testflows/rbac/requirements/requirements.py @@ -3333,9 +3333,9 @@ RQ_SRS_006_RBAC_Quota_Create_Interval = Requirement( '`FOR INTERVAL` clause in the `CREATE QUOTA` statement.\n' '\n' 'This statement SHALL also support a number and a time period which will be one\n' - 'of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be:\n' + 'of `{SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}`. Thus, the complete syntax SHALL be:\n' '\n' - '`FOR INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some real number\n' + '`FOR INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}` where number is some real number\n' 'to define the interval.\n' ), link=None @@ -3354,9 +3354,9 @@ RQ_SRS_006_RBAC_Quota_Create_Interval_Randomized = Requirement( '`FOR RANDOMIZED INTERVAL` clause in the `CREATE QUOTA` statement.\n' '\n' 'This statement SHALL also support a number and a time period which will be one\n' - 'of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be:\n' + 'of `{SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}`. Thus, the complete syntax SHALL be:\n' '\n' - '`FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some\n' + '`FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}` where number is some\n' 'real number to define the interval.\n' ), link=None @@ -3590,7 +3590,7 @@ RQ_SRS_006_RBAC_Quota_Create_Syntax = Requirement( '```sql\n' 'CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name]\n' " [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]\n" - ' [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}\n' + ' [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}\n' ' {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] |\n' ' NO LIMITS | TRACKING ONLY} [,...]]\n' ' [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]\n' @@ -3669,9 +3669,9 @@ RQ_SRS_006_RBAC_Quota_Alter_Interval = Requirement( '`FOR INTERVAL` clause in the `ALTER QUOTA` statement.\n' '\n' 'This statement SHALL also support a number and a time period which will be one\n' - 'of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be:\n' + 'of `{SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}`. Thus, the complete syntax SHALL be:\n' '\n' - '`FOR INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some real number\n' + '`FOR INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}` where number is some real number\n' 'to define the interval.\n' ), link=None @@ -3690,9 +3690,9 @@ RQ_SRS_006_RBAC_Quota_Alter_Interval_Randomized = Requirement( '`FOR RANDOMIZED INTERVAL` clause in the `ALTER QUOTA` statement.\n' '\n' 'This statement SHALL also support a number and a time period which will be one\n' - 'of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be:\n' + 'of `{SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}`. Thus, the complete syntax SHALL be:\n' '\n' - '`FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some\n' + '`FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}` where number is some\n' 'real number to define the interval.\n' ), link=None diff --git a/tests/testflows/rbac/tests/syntax/alter_quota.py b/tests/testflows/rbac/tests/syntax/alter_quota.py index 51f4ca264c0..cec091766ed 100755 --- a/tests/testflows/rbac/tests/syntax/alter_quota.py +++ b/tests/testflows/rbac/tests/syntax/alter_quota.py @@ -15,7 +15,7 @@ def feature(self, node="clickhouse1"): ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name] [RENAME TO new_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | MONTH} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] @@ -93,7 +93,7 @@ def feature(self, node="clickhouse1"): with When("I alter quota on a randomized interval"): node.query("ALTER QUOTA quota0 FOR RANDOMIZED INTERVAL 1 DAY NO LIMITS") - intervals = ['SECOND', 'MINUTE', 'HOUR', 'DAY', 'MONTH'] + intervals = ['SECOND', 'MINUTE', 'HOUR', 'DAY', 'WEEK', 'MONTH', 'QUARTER', 'YEAR'] for i, interval in enumerate(intervals): with Scenario(f"I alter quota for interval {interval}", flags=TE, requirements=[ RQ_SRS_006_RBAC_Quota_Alter_Interval("1.0")]): diff --git a/tests/testflows/rbac/tests/syntax/create_quota.py b/tests/testflows/rbac/tests/syntax/create_quota.py index 51e19b7f793..35013d0b022 100755 --- a/tests/testflows/rbac/tests/syntax/create_quota.py +++ b/tests/testflows/rbac/tests/syntax/create_quota.py @@ -14,7 +14,7 @@ def feature(self, node="clickhouse1"): ```sql CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | NO LIMITS | TRACKING ONLY} [,...]] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] @@ -107,7 +107,7 @@ def feature(self, node="clickhouse1"): with When("I create a quota for randomized interval"): node.query("CREATE QUOTA quota9 FOR RANDOMIZED INTERVAL 1 DAY NO LIMITS") - intervals = ['SECOND', 'MINUTE', 'HOUR', 'DAY', 'MONTH'] + intervals = ['SECOND', 'MINUTE', 'HOUR', 'DAY', 'WEEK', 'MONTH', 'QUARTER', 'YEAR'] for i, interval in enumerate(intervals): with Scenario(f"I create quota for interval {interval}", flags=TE, requirements=[ RQ_SRS_006_RBAC_Quota_Create_Interval("1.0")]): From 306aefd33e12b91984c7d2d8e7ad4ad71c304caa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 17:05:34 +0300 Subject: [PATCH 187/374] Change error code #13416 --- src/Storages/MergeTree/registerStorageMergeTree.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index b7881694565..b59ab15ad74 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -33,7 +33,6 @@ namespace ErrorCodes extern const int NO_ELEMENTS_IN_CONFIG; extern const int UNKNOWN_STORAGE; extern const int NO_REPLICA_NAME_GIVEN; - extern const int LOGICAL_ERROR; } @@ -441,7 +440,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) replica_name = "{replica}"; /// TODO maybe use hostname if {replica} is not defined? } else - throw Exception("Expected zookeper_path and replica_name arguments", ErrorCodes::LOGICAL_ERROR); + throw Exception("Expected zookeper_path and replica_name arguments", ErrorCodes::BAD_ARGUMENTS); /// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries bool is_on_cluster = args.local_context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; From ba204fe3b293aff28113d27c21533bd3bc605682 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 9 Aug 2020 00:24:43 +0800 Subject: [PATCH 188/374] fix test and add alias to FROM_UNIXTIME --- src/Functions/formatDateTime.cpp | 1 + tests/queries/0_stateless/01411_from_unixtime.reference | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index 58e441bd42b..a8abfe7c562 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -718,6 +718,7 @@ void registerFunctionFormatDateTime(FunctionFactory & factory) { factory.registerFunction(); factory.registerFunction(); + factory.registerAlias("fromUnixTimestamp", "FROM_UNIXTIME"); } } diff --git a/tests/queries/0_stateless/01411_from_unixtime.reference b/tests/queries/0_stateless/01411_from_unixtime.reference index 8541fa3f1a0..1bc7519e668 100644 --- a/tests/queries/0_stateless/01411_from_unixtime.reference +++ b/tests/queries/0_stateless/01411_from_unixtime.reference @@ -1,6 +1,6 @@ 1970-01-01 00:02:03 1973-11-29 21:33:09 -2038-07-12 01:15:36 +2106-02-07 06:28:15 19 11 1970-01-15 From 5401694138cacfe76c1f609a494cd406df7bc45a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 7 Aug 2020 22:44:53 +0300 Subject: [PATCH 189/374] splitInto() now can put result into a vector of std::string_view. --- base/common/find_symbols.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/common/find_symbols.h b/base/common/find_symbols.h index 65d4a53ceff..a5921b813a1 100644 --- a/base/common/find_symbols.h +++ b/base/common/find_symbols.h @@ -308,7 +308,7 @@ inline void splitInto(To & to, const std::string & what, bool token_compress = f const char * delimiter_or_end = find_first_symbols(pos, end); if (!token_compress || pos < delimiter_or_end) - to.emplace_back(pos, delimiter_or_end); + to.emplace_back(pos, delimiter_or_end - pos); if (delimiter_or_end < end) pos = delimiter_or_end + 1; From d9db0a34426294565a699b24c9ff698f36e3d0d7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 21:38:34 +0300 Subject: [PATCH 190/374] Add simple "clickhouse-install" tool --- base/CMakeLists.txt | 1 + .../readpassphrase/CMakeLists.txt | 1 + .../readpassphrase/readpassphrase.c | 4 +- .../readpassphrase/readpassphrase.h | 0 .../client => base}/readpassphrase/ya.make | 0 base/ya.make | 1 + programs/CMakeLists.txt | 9 +- programs/client/CMakeLists.txt | 1 - programs/client/ConnectionParameters.cpp | 2 +- programs/client/readpassphrase/includes.h | 9 - programs/config_tools.h.in | 1 + programs/install/CMakeLists.txt | 12 + programs/install/Install.cpp | 506 ++++++++++++++++++ programs/main.cpp | 6 + programs/ya.make | 2 +- 15 files changed, 538 insertions(+), 17 deletions(-) rename {programs/client => base}/readpassphrase/CMakeLists.txt (88%) rename {programs/client => base}/readpassphrase/readpassphrase.c (99%) rename {programs/client => base}/readpassphrase/readpassphrase.h (100%) rename {programs/client => base}/readpassphrase/ya.make (100%) delete mode 100644 programs/client/readpassphrase/includes.h create mode 100644 programs/install/CMakeLists.txt create mode 100644 programs/install/Install.cpp diff --git a/base/CMakeLists.txt b/base/CMakeLists.txt index cfa54fe2ca4..46bd57eda12 100644 --- a/base/CMakeLists.txt +++ b/base/CMakeLists.txt @@ -7,6 +7,7 @@ add_subdirectory (daemon) add_subdirectory (loggers) add_subdirectory (pcg-random) add_subdirectory (widechar_width) +add_subdirectory (readpassphrase) if (USE_MYSQL) add_subdirectory (mysqlxx) diff --git a/programs/client/readpassphrase/CMakeLists.txt b/base/readpassphrase/CMakeLists.txt similarity index 88% rename from programs/client/readpassphrase/CMakeLists.txt rename to base/readpassphrase/CMakeLists.txt index dd1bf2c91b9..574130ce6e3 100644 --- a/programs/client/readpassphrase/CMakeLists.txt +++ b/base/readpassphrase/CMakeLists.txt @@ -5,3 +5,4 @@ add_library(readpassphrase readpassphrase.c) set_target_properties(readpassphrase PROPERTIES LINKER_LANGUAGE C) target_compile_options(readpassphrase PRIVATE -Wno-unused-result -Wno-reserved-id-macro) +target_include_directories(readpassphrase PUBLIC .) diff --git a/programs/client/readpassphrase/readpassphrase.c b/base/readpassphrase/readpassphrase.c similarity index 99% rename from programs/client/readpassphrase/readpassphrase.c rename to base/readpassphrase/readpassphrase.c index 21b48e7efc3..9e8097643bb 100644 --- a/programs/client/readpassphrase/readpassphrase.c +++ b/base/readpassphrase/readpassphrase.c @@ -23,7 +23,9 @@ /* OPENBSD ORIGINAL: lib/libc/gen/readpassphrase.c */ -#include "includes.h" +#ifndef _PATH_TTY +#define _PATH_TTY "/dev/tty" +#endif #include #include diff --git a/programs/client/readpassphrase/readpassphrase.h b/base/readpassphrase/readpassphrase.h similarity index 100% rename from programs/client/readpassphrase/readpassphrase.h rename to base/readpassphrase/readpassphrase.h diff --git a/programs/client/readpassphrase/ya.make b/base/readpassphrase/ya.make similarity index 100% rename from programs/client/readpassphrase/ya.make rename to base/readpassphrase/ya.make diff --git a/base/ya.make b/base/ya.make index 004da9af2ae..bbd961d02c3 100644 --- a/base/ya.make +++ b/base/ya.make @@ -4,4 +4,5 @@ RECURSE( loggers pcg-random widechar_width + readpassphrase ) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index b36a2ff8194..15223af7ede 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -17,6 +17,7 @@ option (ENABLE_CLICKHOUSE_COPIER "Enable clickhouse-copier" ${ENABLE_CLICKHOUSE_ option (ENABLE_CLICKHOUSE_FORMAT "Enable clickhouse-format" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_OBFUSCATOR "Enable clickhouse-obfuscator" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_ODBC_BRIDGE "Enable clickhouse-odbc-bridge" ${ENABLE_CLICKHOUSE_ALL}) +option (ENABLE_CLICKHOUSE_INSTALL "Enable clickhouse-install" ${ENABLE_CLICKHOUSE_ALL}) if(NOT (MAKE_STATIC_LIBRARIES OR SPLIT_SHARED_LIBRARIES)) set(CLICKHOUSE_ONE_SHARED 1) @@ -84,6 +85,7 @@ add_subdirectory (compressor) add_subdirectory (copier) add_subdirectory (format) add_subdirectory (obfuscator) +add_subdirectory (install) if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) add_subdirectory (odbc-bridge) @@ -142,6 +144,9 @@ else () if (ENABLE_CLICKHOUSE_OBFUSCATOR) clickhouse_target_link_split_lib(clickhouse obfuscator) endif () + if (ENABLE_CLICKHOUSE_INSTALL) + clickhouse_target_link_split_lib(clickhouse install) + endif () set (CLICKHOUSE_BUNDLE) if (ENABLE_CLICKHOUSE_SERVER) @@ -202,10 +207,6 @@ else () endif() endif () -if (TARGET clickhouse-server AND TARGET copy-headers) - add_dependencies(clickhouse-server copy-headers) -endif () - if (ENABLE_TESTS AND USE_GTEST) set (CLICKHOUSE_UNIT_TESTS_TARGETS unit_tests_libcommon unit_tests_dbms) add_custom_target (clickhouse-tests ALL DEPENDS ${CLICKHOUSE_UNIT_TESTS_TARGETS}) diff --git a/programs/client/CMakeLists.txt b/programs/client/CMakeLists.txt index 8ec91a10257..72b5caf9784 100644 --- a/programs/client/CMakeLists.txt +++ b/programs/client/CMakeLists.txt @@ -17,7 +17,6 @@ set (CLICKHOUSE_CLIENT_LINK ) # Always use internal readpassphrase -add_subdirectory(readpassphrase) list(APPEND CLICKHOUSE_CLIENT_LINK PRIVATE readpassphrase) clickhouse_program_add(client) diff --git a/programs/client/ConnectionParameters.cpp b/programs/client/ConnectionParameters.cpp index d8b4d0f1add..f0ef3ae5694 100644 --- a/programs/client/ConnectionParameters.cpp +++ b/programs/client/ConnectionParameters.cpp @@ -9,7 +9,7 @@ #include #include #include -#include "readpassphrase/readpassphrase.h" +#include namespace DB { diff --git a/programs/client/readpassphrase/includes.h b/programs/client/readpassphrase/includes.h deleted file mode 100644 index 3ca5eb2bff8..00000000000 --- a/programs/client/readpassphrase/includes.h +++ /dev/null @@ -1,9 +0,0 @@ -#pragma once - -/* #undef HAVE_READPASSPHRASE */ - -#if !defined(HAVE_READPASSPHRASE) -# ifndef _PATH_TTY -# define _PATH_TTY "/dev/tty" -# endif -#endif diff --git a/programs/config_tools.h.in b/programs/config_tools.h.in index ff0a62d8171..11386aca60e 100644 --- a/programs/config_tools.h.in +++ b/programs/config_tools.h.in @@ -12,4 +12,5 @@ #cmakedefine01 ENABLE_CLICKHOUSE_COMPRESSOR #cmakedefine01 ENABLE_CLICKHOUSE_FORMAT #cmakedefine01 ENABLE_CLICKHOUSE_OBFUSCATOR +#cmakedefine01 ENABLE_CLICKHOUSE_INSTALL #cmakedefine01 ENABLE_CLICKHOUSE_ODBC_BRIDGE diff --git a/programs/install/CMakeLists.txt b/programs/install/CMakeLists.txt new file mode 100644 index 00000000000..4b9817e0d37 --- /dev/null +++ b/programs/install/CMakeLists.txt @@ -0,0 +1,12 @@ +set (CLICKHOUSE_INSTALL_SOURCES Install.cpp) + +set (CLICKHOUSE_INSTALL_LINK + PRIVATE + boost::program_options + clickhouse_common_io + dbms + readpassphrase +) + +clickhouse_program_add(install) + diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp new file mode 100644 index 00000000000..cb292bcbd0b --- /dev/null +++ b/programs/install/Install.cpp @@ -0,0 +1,506 @@ +#include +#include +#include + +#include + +#if defined(__linux__) + #include + #include +#endif + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + + +/** This tool can be used to install ClickHouse without a deb/rpm/tgz package, having only "clickhouse" binary. + * The following steps are performed: + * + * - copying the binary to binary directory (/usr/bin). + * - creation of symlinks for tools. + * - creation of clickhouse user and group. + * - creation of config directory (/etc/clickhouse-server). + * - creation of default configuration files. + * - creation of a directory for logs (/var/log/clickhouse-server). + * - creation of a data directory if not exists. + * - setting a password for default user. + * - choose an option to listen connections. + * - changing the ownership and mode of the directories. + * - setting capabilities for binary. + * - setting ulimits for the user. + * + * It does not install clickhouse-odbc-bridge. + */ + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FILE_DOESNT_EXIST; + extern const int CANNOT_OPEN_FILE; + extern const int SYSTEM_ERROR; + extern const int NOT_ENOUGH_SPACE; +} + +void executeScript(const std::string & command) +{ + auto sh = ShellCommand::execute(command); + WriteBufferFromFileDescriptor stdout(STDOUT_FILENO); + WriteBufferFromFileDescriptor stderr(STDERR_FILENO); + copyData(sh->out, stdout); + copyData(sh->err, stderr); + sh->tryWait(); +} + +bool ask(std::string question) +{ + while (true) + { + std::string answer; + std::cout << question; + std::getline(std::cin, answer); + if (!std::cin.good()) + return false; + + if (answer.empty() || answer == "n" || answer == "N") + return false; + if (answer == "y" || answer == "Y") + return true; + } +} + +} + + +int mainEntryClickHouseInstall(int argc, char ** argv) +{ + using namespace DB; + namespace po = boost::program_options; + namespace fs = std::filesystem; + + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("prefix", po::value()->default_value(""), "prefix for all paths") + ("binary-path", po::value()->default_value("/usr/bin"), "where to install binaries") + ("config-path", po::value()->default_value("/etc/clickhouse-server"), "where to install configs") + ("log-path", po::value()->default_value("/var/log/clickhouse-server"), "where to create log directory") + ("data-path", po::value()->default_value("/var/lib/clickhouse"), "directory for data") + ("user", po::value()->default_value("clickhouse"), "clickhouse user to create") + ("group", po::value()->default_value("clickhouse"), "clickhouse group to create") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " install [options]\n"; + std::cout << desc << '\n'; + return 1; + } + + try + { + /// We need to copy binary to the binary directory. + /// The binary is currently run. We need to obtain its path from procfs. + + fs::path binary_self_path = "/proc/self/exe"; + if (!fs::exists(binary_self_path)) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot obtain path to the binary from {}, file doesn't exist", + binary_self_path.string()); + + /// Copy binary to the destination directory. + + size_t binary_size = fs::file_size(binary_self_path); + + fs::path prefix = fs::path(options["prefix"].as()); + fs::path bin_dir = prefix / fs::path(options["binary-path"].as()); + + size_t available_space = fs::space(bin_dir).available; + if (available_space < binary_size) + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space for clickhouse binary in {}, required {}, available {}.", + bin_dir.string(), ReadableSize(binary_size), ReadableSize(available_space)); + + fs::path main_bin_path = bin_dir / "clickhouse"; + fs::path main_bin_tmp_path = bin_dir / "clickhouse.new"; + fs::path main_bin_old_path = bin_dir / "clickhouse.old"; + + fmt::print("Copying ClickHouse binary to {}\n", main_bin_tmp_path.string()); + + try + { + ReadBufferFromFile in(binary_self_path.string()); + WriteBufferFromFile out(main_bin_tmp_path.string()); + copyData(in, out); + out.sync(); + + if (0 != fchmod(out.getFD(), S_IRUSR | S_IRGRP | S_IROTH | S_IXUSR | S_IXGRP | S_IXOTH)) + throwFromErrno(fmt::format("Cannot chmod {}", main_bin_tmp_path.string()), ErrorCodes::SYSTEM_ERROR); + + out.finalize(); + } + catch (const Exception & e) + { + if (e.code() == ErrorCodes::CANNOT_OPEN_FILE && geteuid() != 0) + std::cerr << "Install must be run as root: sudo ./clickhouse install"; + throw; + } + + if (fs::exists(main_bin_path)) + { + fmt::print("{} already exists, will rename existing binary to {} and put the new binary in place\n", + main_bin_path.string(), main_bin_old_path.string()); + + /// There is file exchange operation in Linux but it's not portable. + fs::rename(main_bin_path, main_bin_old_path); + } + + fmt::print("Renaming {} to {}.\n", main_bin_tmp_path.string(), main_bin_path.string()); + fs::rename(main_bin_tmp_path, main_bin_path); + + /// Create symlinks. + + std::initializer_list tools + { + "clickhouse-server", + "clickhouse-client", + "clickhouse-local", + "clickhouse-benchmark", + "clickhouse-copier", + "clickhouse-obfuscator", + "clickhouse-compressor", + "clickhouse-format", + "clickhouse-extract-from-config" + }; + + for (const auto & tool : tools) + { + bool need_to_create = true; + fs::path symlink_path = bin_dir / tool; + + if (fs::exists(symlink_path)) + { + bool is_symlink = fs::is_symlink(symlink_path); + fs::path points_to; + if (is_symlink) + points_to = fs::absolute(fs::read_symlink(symlink_path)); + + if (is_symlink && points_to == main_bin_path) + { + need_to_create = false; + } + else + { + if (!is_symlink) + { + fs::path rename_path = symlink_path.replace_extension(".old"); + fmt::print("File {} already exists but it's not a symlink. Will rename to {}.\n", + symlink_path.string(), rename_path.string()); + fs::rename(symlink_path, rename_path); + } + else if (points_to != main_bin_path) + { + fmt::print("Symlink {} already exists but it points to {}. Will replace the old symlink to {}.\n", + symlink_path.string(), points_to.string(), main_bin_path.string()); + fs::remove(symlink_path); + } + } + } + + if (need_to_create) + { + fmt::print("Creating symlink {} to {}.\n", symlink_path.string(), main_bin_path.string()); + fs::create_symlink(main_bin_path, symlink_path); + } + } + + /// Creation of clickhouse user and group. + + std::string user = options["user"].as(); + std::string group = options["group"].as(); + + if (!group.empty()) + { + { + fmt::print("Creating clickhouse group if it does not exist.\n"); + std::string command = fmt::format("groupadd -r {}", group); + fmt::print(" {}\n", command); + executeScript(command); + } + } + else + fmt::print("Will not create clickhouse group"); + + if (!user.empty()) + { + fmt::print("Creating clickhouse user if it does not exist.\n"); + std::string command = group.empty() + ? fmt::format("useradd --system --shell /bin/false --home-dir /nonexistent --user-group {}", user) + : fmt::format("useradd --system --shell /bin/false --home-dir /nonexistent -g {} {}", group, user); + fmt::print(" {}\n", command); + executeScript(command); + + /// Setting ulimits. + try + { + fs::path ulimits_dir = "/etc/security/limits.d"; + fs::path ulimits_file = ulimits_dir / fmt::format("{}.conf", user); + fmt::print("Will set ulimits for {} user in {}.\n", user, ulimits_file.string()); + std::string ulimits_content = fmt::format( + "{0}\tsoft\tnofile\t262144\n" + "{0}\thard\tnofile\t262144\n", user); + + fs::create_directories(ulimits_dir); + + WriteBufferFromFile out(ulimits_file.string()); + out.write(ulimits_content.data(), ulimits_content.size()); + out.sync(); + out.finalize(); + } + catch (...) + { + std::cerr << "Cannot set ulimits: " << getCurrentExceptionMessage(false) << "\n"; + } + + /// TODO Set ulimits on Mac OS X + } + else + fmt::print("Will not create clickhouse user.\n"); + + /// Creating configuration files and directories. + + fs::path config_dir = prefix / options["config-path"].as(); + + if (!fs::exists(config_dir)) + { + fmt::print("Creating config directory {}.\n", config_dir.string()); + fs::create_directories(config_dir); + } + + fs::path main_config_file = config_dir / "config.xml"; + fs::path users_config_file = config_dir / "users.xml"; + fs::path config_d = config_dir / "config.d"; + fs::path users_d = config_dir / "users.d"; + + std::string log_path = prefix / options["log-path"].as(); + std::string data_path = prefix / options["data-path"].as(); + + bool has_password_for_default_user = false; + + if (!fs::exists(main_config_file)) + { + fmt::print("Creating config directory {} that is used for tweaks of main server configuration.\n", config_d.string()); + fs::create_directory(config_d); + + fmt::print("Creating config directory {} that is used for tweaks of users configuration.\n", users_d.string()); + fs::create_directory(users_d); + + std::string_view main_config_content = getResource("config.xml"); + if (main_config_content.empty()) + { + fmt::print("There is no default config.xml, you have to download it and place to {}.\n", main_config_file.string()); + } + else + { + WriteBufferFromFile out(main_config_file.string()); + out.write(main_config_content.data(), main_config_content.size()); + out.sync(); + out.finalize(); + } + + std::string_view users_config_content = getResource("users.xml"); + if (users_config_content.empty()) + { + fmt::print("There is no default users.xml, you have to download it and place to {}.\n", users_config_file.string()); + } + else + { + WriteBufferFromFile out(users_config_file.string()); + out.write(users_config_content.data(), users_config_content.size()); + out.sync(); + out.finalize(); + } + + /// TODO chmod configs? + } + else + { + { + fmt::print("Config file {} already exists, will keep it and extract path info from it.\n", main_config_file.string()); + + ConfigProcessor processor(main_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); + ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(processor.processConfig())); + + if (configuration->has("path")) + { + data_path = configuration->getString("path"); + fmt::print("{} has {} as data path.\n", main_config_file.string(), data_path); + } + + if (configuration->has("logger.log")) + { + log_path = fs::path(configuration->getString("logger.log")).remove_filename(); + fmt::print("{} has {} as log path.\n", main_config_file.string(), log_path); + } + } + + /// Check if password for default user already specified. + + if (fs::exists(users_config_file)) + { + ConfigProcessor processor(users_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); + ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(processor.processConfig())); + + if (!configuration->getString("users.default.password", "").empty() + || configuration->getString("users.default.password_sha256_hex", "").empty() + || configuration->getString("users.default.password_double_sha1_hex", "").empty()) + { + has_password_for_default_user = true; + } + } + } + + /// Create directories for data and log. + + if (fs::exists(log_path)) + { + fmt::print("Log directory {} already exists.\n", log_path); + } + else + { + fmt::print("Creating log directory {}.\n", log_path); + fs::create_directories(log_path); + } + + if (fs::exists(data_path)) + { + fmt::print("Data directory {} already exists.\n", data_path); + } + else + { + fmt::print("Creating data directory {}.\n", data_path); + fs::create_directories(data_path); + } + + /// TODO chmod dirs + + /// Set up password for default user. + + bool stdin_is_a_tty = isatty(STDIN_FILENO); + bool stdout_is_a_tty = isatty(STDOUT_FILENO); + bool is_interactive = stdin_is_a_tty && stdout_is_a_tty; + + if (has_password_for_default_user) + { + fmt::print("Password for default user is already specified. To remind or reset, see {} and {}.\n", + users_config_file.string(), users_d.string()); + } + else if (!is_interactive) + { + fmt::print("Password for default user is empty string. See {} and {} to change it.\n", + users_config_file.string(), users_d.string()); + } + else + { + char buf[1000] = {}; + std::string password; + if (auto * result = readpassphrase("Enter password for default user: ", buf, sizeof(buf), 0)) + password = result; + + if (!password.empty()) + { + std::string password_file = users_d / "default-password.xml"; + WriteBufferFromFile out(password_file); +#if USE_SSL + std::vector hash; + hash.resize(32); + encodeSHA256(password, hash.data()); + std::string hash_hex; + hash_hex.resize(64); + for (size_t i = 0; i < 32; ++i) + writeHexByteLowercase(hash[i], &hash_hex[2 * i]); + out << "\n" + " \n" + " \n" + " " << hash_hex << "\n" + " \n" + " \n" + "\n"; + out.sync(); + out.finalize(); + fmt::print("Password for default user is saved in file {}.\n", password_file); +#else + out << "\n" + " \n" + " \n" + " \n" + " \n" + " \n" + "\n"; + out.sync(); + out.finalize(); + fmt::print("Password for default user is saved in plaintext in file {}.\n", password_file); +#endif + has_password_for_default_user = true; + } + else + fmt::print("Password for default user is empty string. See {} and {} to change it.\n", + users_config_file.string(), users_d.string()); + } + + /// Set capabilities for the binary. + +#if defined(__linux__) + fmt::print("Setting capabilities for clickhouse binary. This is optional.\n"); + std::string command = fmt::format("command setcap && setcap 'cap_net_admin,cap_ipc_lock,cap_sys_nice+ep' {}", main_bin_path.string()); + fmt::print(" {}\n", command); + executeScript(command); +#endif + + /// If password was set, ask for open for connections. + if (is_interactive && has_password_for_default_user) + { + if (ask("Allow server to accept connections from the network (default is localhost only), [y/N]: ")) + { + std::string listen_file = config_d / "listen.xml"; + WriteBufferFromFile out(listen_file); + out << "\n" + " ::\n" + "\n"; + out.sync(); + out.finalize(); + fmt::print("The choice is saved in file {}.\n", listen_file); + } + } + + fmt::print("\nClickHouse has been successfully installed.\n"); + } + catch (...) + { + std::cerr << getCurrentExceptionMessage(false); + return getCurrentExceptionCode(); + } + + return 0; +} + diff --git a/programs/main.cpp b/programs/main.cpp index 382a104b798..6f6e706bce0 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -46,6 +46,9 @@ int mainEntryClickHouseClusterCopier(int argc, char ** argv); #if ENABLE_CLICKHOUSE_OBFUSCATOR int mainEntryClickHouseObfuscator(int argc, char ** argv); #endif +#if ENABLE_CLICKHOUSE_INSTALL +int mainEntryClickHouseInstall(int argc, char ** argv); +#endif namespace @@ -84,6 +87,9 @@ std::pair clickhouse_applications[] = #if ENABLE_CLICKHOUSE_OBFUSCATOR {"obfuscator", mainEntryClickHouseObfuscator}, #endif +#if ENABLE_CLICKHOUSE_INSTALL + {"install", mainEntryClickHouseInstall}, +#endif }; diff --git a/programs/ya.make b/programs/ya.make index 7c967470640..fac3d128033 100644 --- a/programs/ya.make +++ b/programs/ya.make @@ -9,7 +9,7 @@ CFLAGS( PEERDIR( clickhouse/base/daemon clickhouse/base/loggers - clickhouse/programs/client/readpassphrase + clickhouse/base/readpassphrase clickhouse/src ) From e5c3f47c92a2c5fa8ce73e73a9f1effc87e64b06 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Aug 2020 23:42:10 +0300 Subject: [PATCH 191/374] Added start/stop/status/restart commands --- programs/install/Install.cpp | 421 +++++++++++++++++++++++++++++++++-- programs/main.cpp | 8 + 2 files changed, 415 insertions(+), 14 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index cb292bcbd0b..b4f9173020d 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -29,6 +29,8 @@ /** This tool can be used to install ClickHouse without a deb/rpm/tgz package, having only "clickhouse" binary. + * It also allows to avoid dependency on systemd, upstart, SysV init. + * * The following steps are performed: * * - copying the binary to binary directory (/usr/bin). @@ -43,6 +45,7 @@ * - changing the ownership and mode of the directories. * - setting capabilities for binary. * - setting ulimits for the user. + * - (todo) put service in cron. * * It does not install clickhouse-odbc-bridge. */ @@ -58,14 +61,29 @@ namespace ErrorCodes extern const int NOT_ENOUGH_SPACE; } -void executeScript(const std::string & command) +} + + +using namespace DB; +namespace po = boost::program_options; +namespace fs = std::filesystem; + + +auto executeScript(const std::string & command, bool throw_on_error = false) { auto sh = ShellCommand::execute(command); WriteBufferFromFileDescriptor stdout(STDOUT_FILENO); WriteBufferFromFileDescriptor stderr(STDERR_FILENO); copyData(sh->out, stdout); copyData(sh->err, stderr); - sh->tryWait(); + + if (throw_on_error) + { + sh->wait(); + return 0; + } + else + return sh->tryWait(); } bool ask(std::string question) @@ -85,15 +103,9 @@ bool ask(std::string question) } } -} - int mainEntryClickHouseInstall(int argc, char ** argv) { - using namespace DB; - namespace po = boost::program_options; - namespace fs = std::filesystem; - po::options_description desc; desc.add_options() ("help,h", "produce help message") @@ -102,6 +114,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) ("config-path", po::value()->default_value("/etc/clickhouse-server"), "where to install configs") ("log-path", po::value()->default_value("/var/log/clickhouse-server"), "where to create log directory") ("data-path", po::value()->default_value("/var/lib/clickhouse"), "directory for data") + ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") ("user", po::value()->default_value("clickhouse"), "clickhouse user to create") ("group", po::value()->default_value("clickhouse"), "clickhouse group to create") ; @@ -255,11 +268,14 @@ int mainEntryClickHouseInstall(int argc, char ** argv) { fmt::print("Creating clickhouse user if it does not exist.\n"); std::string command = group.empty() - ? fmt::format("useradd --system --shell /bin/false --home-dir /nonexistent --user-group {}", user) - : fmt::format("useradd --system --shell /bin/false --home-dir /nonexistent -g {} {}", group, user); + ? fmt::format("useradd -r --shell /bin/false --home-dir /nonexistent --user-group {}", user) + : fmt::format("useradd -r --shell /bin/false --home-dir /nonexistent -g {} {}", group, user); fmt::print(" {}\n", command); executeScript(command); + if (group.empty()) + group = user; + /// Setting ulimits. try { @@ -304,6 +320,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) std::string log_path = prefix / options["log-path"].as(); std::string data_path = prefix / options["data-path"].as(); + std::string pid_path = prefix / options["pid-path"].as(); bool has_password_for_default_user = false; @@ -340,8 +357,6 @@ int mainEntryClickHouseInstall(int argc, char ** argv) out.sync(); out.finalize(); } - - /// TODO chmod configs? } else { @@ -380,6 +395,28 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } } + /// Chmod and chown configs + { + std::string command = fmt::format("chown --recursive {}:{} '{}'", user, group, config_dir.string()); + fmt::print(" {}\n", command); + executeScript(command); + } + + /// Symlink "preprocessed_configs" is created by the server, so "write" is needed. + fs::permissions(config_dir, fs::perms::owner_all, fs::perm_options::replace); + + /// Subdirectories, so "execute" is needed. + if (fs::exists(config_d)) + fs::permissions(config_d, fs::perms::owner_read | fs::perms::owner_exec, fs::perm_options::replace); + if (fs::exists(users_d)) + fs::permissions(users_d, fs::perms::owner_read | fs::perms::owner_exec, fs::perm_options::replace); + + /// Readonly. + if (fs::exists(main_config_file)) + fs::permissions(main_config_file, fs::perms::owner_read, fs::perm_options::replace); + if (fs::exists(users_config_file)) + fs::permissions(users_config_file, fs::perms::owner_read, fs::perm_options::replace); + /// Create directories for data and log. if (fs::exists(log_path)) @@ -402,7 +439,41 @@ int mainEntryClickHouseInstall(int argc, char ** argv) fs::create_directories(data_path); } - /// TODO chmod dirs + if (fs::exists(pid_path)) + { + fmt::print("Pid directory {} already exists.\n", pid_path); + } + else + { + fmt::print("Creating pid directory {}.\n", pid_path); + fs::create_directories(pid_path); + } + + /// Chmod and chown data and log directories + { + std::string command = fmt::format("chown --recursive {}:{} '{}'", user, group, log_path); + fmt::print(" {}\n", command); + executeScript(command); + } + + { + std::string command = fmt::format("chown --recursive {}:{} '{}'", user, group, pid_path); + fmt::print(" {}\n", command); + executeScript(command); + } + + { + /// Not recursive, because there can be a huge number of files and it will be slow. + std::string command = fmt::format("chown {}:{} '{}'", user, group, data_path); + fmt::print(" {}\n", command); + executeScript(command); + } + + /// Other users in clickhouse group are allowed to read and even delete logs. + fs::permissions(log_path, fs::perms::owner_all | fs::perms::group_all, fs::perm_options::replace); + + /// Data directory is not accessible to anyone except clickhouse. + fs::permissions(data_path, fs::perms::owner_all, fs::perm_options::replace); /// Set up password for default user. @@ -442,6 +513,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) out << "\n" " \n" " \n" + " \n" " " << hash_hex << "\n" " \n" " \n" @@ -493,7 +565,21 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } } - fmt::print("\nClickHouse has been successfully installed.\n"); + std::string maybe_sudo; + if (getuid() != 0) + maybe_sudo = "sudo "; + + std::string maybe_password; + if (has_password_for_default_user) + maybe_password = " --password"; + + fmt::print( + "\nClickHouse has been successfully installed.\n" + "\nStart clickhouse-server with:\n" + " {}clickhouse start\n" + "\nStart clickhouse-client with:\n" + " clickhouse-client{}\n\n", + maybe_sudo, maybe_password); } catch (...) { @@ -504,3 +590,310 @@ int mainEntryClickHouseInstall(int argc, char ** argv) return 0; } + +namespace +{ + int start(const std::string & user, const fs::path & executable, const fs::path & config, const fs::path & pid_file) + { + if (fs::exists(pid_file)) + { + ReadBufferFromFile in(pid_file.string()); + UInt64 pid; + if (tryReadIntText(pid, in)) + { + fmt::print("{} file exists and contains pid = {}.\n", pid_file.string(), pid); + + if (0 == kill(pid, 0)) + { + fmt::print("The process with pid = {} is already running.\n", pid); + return 2; + } + } + else + { + fmt::print("{} file exists but damaged, ignoring.\n", pid_file.string()); + fs::remove(pid_file); + } + } + + std::string command = fmt::format("{} --config-file {} --pid-file {} --daemon", + executable.string(), config.string(), pid_file.string()); + + if (!user.empty()) + { + bool need_sudo = geteuid() != 0; + if (need_sudo) + command = fmt::format("sudo -u '{}' {}", user, command); + else + command = fmt::format("su -s /bin/sh '{}' -c '{}'", user, command); + } + + fmt::print("Will run {}\n", command); + executeScript(command, true); + + /// Wait to start. + + size_t try_num = 0; + constexpr size_t num_tries = 60; + for (; try_num < num_tries; ++try_num) + { + fmt::print("Waiting for server to start\n"); + if (fs::exists(pid_file)) + { + fmt::print("Server started\n"); + break; + } + ::sleep(1); + } + + if (try_num == num_tries) + { + fmt::print("Cannot start server. You can execute {} without --daemon option to run manually.\n", command); + return 3; + } + + return 0; + } + + UInt64 isRunning(const fs::path & pid_file) + { + UInt64 pid = 0; + + if (fs::exists(pid_file)) + { + ReadBufferFromFile in(pid_file.string()); + if (tryReadIntText(pid, in)) + { + fmt::print("{} file exists and contains pid = {}.\n", pid_file.string(), pid); + } + else + { + fmt::print("{} file exists but damaged, ignoring.\n", pid_file.string()); + fs::remove(pid_file); + } + } + + if (!pid) + { + auto sh = ShellCommand::execute("pidof clickhouse-server"); + + if (tryReadIntText(pid, sh->out)) + { + fmt::print("Found pid = {} in the list of running processes.\n", pid); + } + else if (!sh->out.eof()) + { + fmt::print("The pidof command returned unusual output.\n"); + } + + WriteBufferFromFileDescriptor stderr(STDERR_FILENO); + copyData(sh->err, stderr); + + sh->tryWait(); + } + + if (pid) + { + if (0 == kill(pid, 0)) + { + fmt::print("The process with pid = {} is running.\n", pid); + } + } + + if (!pid) + { + fmt::print("Now there is no clickhouse-server process.\n"); + } + + return pid; + } + + int stop(const fs::path & pid_file) + { + UInt64 pid = isRunning(pid_file); + + if (!pid) + return 0; + + if (0 == kill(pid, 15)) /// Terminate + fmt::print("Sent termination signal.\n", pid); + else + throwFromErrno("Cannot send termination signal", ErrorCodes::SYSTEM_ERROR); + + size_t try_num = 0; + constexpr size_t num_tries = 60; + for (; try_num < num_tries; ++try_num) + { + fmt::print("Waiting for server to stop\n"); + if (!isRunning(pid_file)) + { + fmt::print("Server stopped\n"); + break; + } + ::sleep(1); + } + + if (try_num == num_tries) + { + fmt::print("Will terminate forcefully.\n", pid); + if (0 == kill(pid, 9)) + fmt::print("Sent kill signal.\n", pid); + else + throwFromErrno("Cannot send kill signal", ErrorCodes::SYSTEM_ERROR); + } + + return 0; + } +} + + +int mainEntryClickHouseStart(int argc, char ** argv) +{ + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("binary-path", po::value()->default_value("/usr/bin"), "directory with binary") + ("config-path", po::value()->default_value("/etc/clickhouse-server"), "directory with configs") + ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") + ("user", po::value()->default_value("clickhouse"), "clickhouse user") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " start\n"; + return 1; + } + + try + { + std::string user = options["user"].as(); + + fs::path executable = fs::path(options["binary-path"].as()) / "clickhouse-server"; + fs::path config = fs::path(options["config-path"].as()) / "config.xml"; + fs::path pid_file = fs::path(options["pid-path"].as()) / "clickhouse-server.pid"; + + return start(user, executable, config, pid_file); + } + catch (...) + { + std::cerr << getCurrentExceptionMessage(false); + return getCurrentExceptionCode(); + } +} + + +int mainEntryClickHouseStop(int argc, char ** argv) +{ + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " stop\n"; + return 1; + } + + try + { + fs::path pid_file = fs::path(options["pid-path"].as()) / "clickhouse-server.pid"; + + return stop(pid_file); + } + catch (...) + { + std::cerr << getCurrentExceptionMessage(false); + return getCurrentExceptionCode(); + } +} + + +int mainEntryClickHouseStatus(int argc, char ** argv) +{ + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " status\n"; + return 1; + } + + try + { + fs::path pid_file = fs::path(options["pid-path"].as()) / "clickhouse-server.pid"; + isRunning(pid_file); + return 0; + } + catch (...) + { + std::cerr << getCurrentExceptionMessage(false); + return getCurrentExceptionCode(); + } +} + + +int mainEntryClickHouseRestart(int argc, char ** argv) +{ + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("binary-path", po::value()->default_value("/usr/bin"), "directory with binary") + ("config-path", po::value()->default_value("/etc/clickhouse-server"), "directory with configs") + ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") + ("user", po::value()->default_value("clickhouse"), "clickhouse user") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " restart\n"; + return 1; + } + + try + { + std::string user = options["user"].as(); + + fs::path executable = fs::path(options["binary-path"].as()) / "clickhouse-server"; + fs::path config = fs::path(options["config-path"].as()) / "config.xml"; + fs::path pid_file = fs::path(options["pid-path"].as()) / "clickhouse-server.pid"; + + if (int res = stop(pid_file)) + return res; + return start(user, executable, config, pid_file); + } + catch (...) + { + std::cerr << getCurrentExceptionMessage(false); + return getCurrentExceptionCode(); + } +} diff --git a/programs/main.cpp b/programs/main.cpp index 6f6e706bce0..3df5f9f683b 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -48,6 +48,10 @@ int mainEntryClickHouseObfuscator(int argc, char ** argv); #endif #if ENABLE_CLICKHOUSE_INSTALL int mainEntryClickHouseInstall(int argc, char ** argv); +int mainEntryClickHouseStart(int argc, char ** argv); +int mainEntryClickHouseStop(int argc, char ** argv); +int mainEntryClickHouseStatus(int argc, char ** argv); +int mainEntryClickHouseRestart(int argc, char ** argv); #endif @@ -89,6 +93,10 @@ std::pair clickhouse_applications[] = #endif #if ENABLE_CLICKHOUSE_INSTALL {"install", mainEntryClickHouseInstall}, + {"start", mainEntryClickHouseStart}, + {"stop", mainEntryClickHouseStop}, + {"status", mainEntryClickHouseStatus}, + {"restart", mainEntryClickHouseRestart}, #endif }; From d2351dc7f3e9bd2d5167142aca9af1a308fa2b1f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Aug 2020 01:50:20 +0300 Subject: [PATCH 192/374] Fix clang static analyzer --- src/Common/tests/average.cpp | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Common/tests/average.cpp b/src/Common/tests/average.cpp index e1a617524b3..3bfe2f0b7f4 100644 --- a/src/Common/tests/average.cpp +++ b/src/Common/tests/average.cpp @@ -106,7 +106,7 @@ Float NO_INLINE baseline_baseline(const PODArray & keys, const PODArrayadd(values[i]); } - return map[0]->result(); + return map[0] ? map[0]->result() : 0; } @@ -125,7 +125,7 @@ Float NO_INLINE baseline(const PODArray & keys, const PODArray & v place->add(values[i]); } - return map[0]->result(); + return map[0] ? map[0]->result() : 0; } @@ -144,7 +144,7 @@ Float NO_INLINE implicit_zero(const PODArray & keys, const PODArrayadd(values[i]); } - return map[0]->result(); + return map[0] ? map[0]->result() : 0; } @@ -170,7 +170,7 @@ Float NO_INLINE calculated_size(const PODArray & keys, const PODArrayadd(values[i]); } - return map[0]->result(); + return map[0] ? map[0]->result() : 0; } @@ -189,7 +189,7 @@ Float NO_INLINE implicit_zero_and_calculated_size(const PODArray & keys, place->add(values[i]); } - return map[0]->result(); + return map[0] ? map[0]->result() : 0; } Float NO_INLINE init_out_of_the_loop(const PODArray & keys, const PODArray & values) @@ -207,7 +207,7 @@ Float NO_INLINE init_out_of_the_loop(const PODArray & keys, const PODArra place->add(values[i]); } - return map[0]->result(); + return map[0] ? map[0]->result() : 0; } Float NO_INLINE embedded_states(const PODArray & keys, const PODArray & values) @@ -239,7 +239,7 @@ Float NO_INLINE simple_lookup_table(const PODArray & keys, const PODArray place->add(values[i]); } - return map[0]->result(); + return map[0] ? map[0]->result() : 0; } Float NO_INLINE simple_lookup_table_embedded_states(const PODArray & keys, const PODArray & values) @@ -288,7 +288,7 @@ Float NO_INLINE unrolled(const PODArray & keys, const PODArray & v place->add(values[i]); } - return map[0]->result(); + return map[0] ? map[0]->result() : 0; } template @@ -326,7 +326,7 @@ Float NO_INLINE simple_lookup_table_unrolled(const PODArray & keys, const place->add(values[i]); } - return map[0]->result(); + return map[0] ? map[0]->result() : 0; } template From d1935bc9677925ccf937f07c76c39f59a92635f2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Aug 2020 02:14:11 +0300 Subject: [PATCH 193/374] Adjust query time in perf test --- tests/performance/synthetic_hardware_benchmark.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/synthetic_hardware_benchmark.xml b/tests/performance/synthetic_hardware_benchmark.xml index 468e92c7aac..64c9aa05501 100644 --- a/tests/performance/synthetic_hardware_benchmark.xml +++ b/tests/performance/synthetic_hardware_benchmark.xml @@ -51,7 +51,7 @@ SELECT number % 10 AS k, count(), sum(number), avg(number) FROM numbers_mt(1600000000) GROUP BY k FORMAT Null SELECT number % 256 AS k, count(), sum(number), avg(number) FROM numbers(100000000) GROUP BY k FORMAT Null SELECT number % 256 AS k, count(), sum(number), avg(number) FROM numbers_mt(1600000000) GROUP BY k FORMAT Null -SELECT number % 256 AS k, count(), sum(number), avg(number), min(number), max(number), uniq(number), any(number), argMin(number, number), argMax(number, number) FROM numbers_mt(160000000) GROUP BY k FORMAT Null +SELECT number % 256 AS k, count(), sum(number), avg(number), min(number), max(number), uniq(number), any(number), argMin(number, number), argMax(number, number) FROM numbers_mt(16000000) GROUP BY k FORMAT Null SELECT number % 1000 AS k, count() FROM numbers( 100000000) GROUP BY k FORMAT Null From 8d4a0243d33f718d501f592f58c356f556624257 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Aug 2020 02:22:56 +0300 Subject: [PATCH 194/374] Fix shellcheck notices --- tests/queries/0_stateless/01195_formats_diagnostic_info.sh | 1 + .../queries/0_stateless/01396_inactive_replica_cleanup_nodes.sh | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01195_formats_diagnostic_info.sh b/tests/queries/0_stateless/01195_formats_diagnostic_info.sh index fce0b0ec974..9e11d04161b 100755 --- a/tests/queries/0_stateless/01195_formats_diagnostic_info.sh +++ b/tests/queries/0_stateless/01195_formats_diagnostic_info.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# shellcheck disable=SC2206 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes.sh index 27fa4261504..4b55bb29c22 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes.sh @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT --max_block_size 1 --min_insert_block_size_rows 1 --min_inser # Now wait for cleanup thread -for i in {1..60}; do +for _ in {1..60}; do $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" [[ $($CLICKHOUSE_CLIENT --query "SELECT sum(toUInt32(extract(message, 'Removed (\d+) old log entries'))) FROM system.text_log WHERE event_date >= yesterday() AND logger_name LIKE '%' || currentDatabase() || '%r1%(ReplicatedMergeTreeCleanupThread)%' AND message LIKE '%Removed % old log entries%'") -gt 9900 ]] && break; sleep 1 From b633b49e072bdca7419f04fb93469ddfaacc5b39 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Aug 2020 02:26:59 +0300 Subject: [PATCH 195/374] Enable Shellcheck in CI --- utils/check-style/check-style | 3 +++ 1 file changed, 3 insertions(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 939eb3ecca8..67f74b7c0d7 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -68,3 +68,6 @@ find $ROOT_PATH -name '.gitmodules' | while read i; do grep -F 'url = ' $i | gre # There shouldn't be any code snippets under GPL or LGPL find $ROOT_PATH/{src,base,programs} -name '*.h' -or -name '*.cpp' 2>/dev/null | xargs grep -i -F 'General Public License' && echo "There shouldn't be any code snippets under GPL or LGPL" + +# Check sh tests with Shellcheck +(cd $ROOT_PATH/tests/queries/0_stateless/ && shellcheck --check-sourced --external-sources --severity info --exclude SC1071,SC2086 *.sh ../1_stateful/*.sh) From b4ac90046262260e7b431328d728733dc978fd93 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Aug 2020 02:38:53 +0300 Subject: [PATCH 196/374] Fix "Arcadia" build --- base/common/ya.make | 5 ++++- base/common/ya.make.in | 36 ++++++++++++++++++++++++++++++++++++ src/Disks/ya.make | 2 ++ 3 files changed, 42 insertions(+), 1 deletion(-) create mode 100644 base/common/ya.make.in diff --git a/base/common/ya.make b/base/common/ya.make index b64ab93f2fc..caa0d067ef2 100644 --- a/base/common/ya.make +++ b/base/common/ya.make @@ -1,3 +1,4 @@ +# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. LIBRARY() ADDINCL( @@ -35,8 +36,10 @@ SRCS( DateLUT.cpp DateLUTImpl.cpp demangle.cpp + errnoToString.cpp getFQDNOrHostName.cpp getMemoryAmount.cpp + getResource.cpp getThreadId.cpp JSON.cpp LineReader.cpp @@ -47,7 +50,7 @@ SRCS( shift10.cpp sleep.cpp terminalColors.cpp - errnoToString.cpp + ) END() diff --git a/base/common/ya.make.in b/base/common/ya.make.in new file mode 100644 index 00000000000..957ae4d68b7 --- /dev/null +++ b/base/common/ya.make.in @@ -0,0 +1,36 @@ +LIBRARY() + +ADDINCL( + GLOBAL clickhouse/base + GLOBAL contrib/libs/cctz/include +) + +CFLAGS (GLOBAL -DARCADIA_BUILD) + +CFLAGS (GLOBAL -DUSE_CPUID=1) +CFLAGS (GLOBAL -DUSE_JEMALLOC=0) +CFLAGS (GLOBAL -DUSE_RAPIDJSON=1) + +IF (OS_DARWIN) + CFLAGS (GLOBAL -DOS_DARWIN) +ELSEIF (OS_FREEBSD) + CFLAGS (GLOBAL -DOS_FREEBSD) +ELSEIF (OS_LINUX) + CFLAGS (GLOBAL -DOS_LINUX) +ENDIF () + +PEERDIR( + contrib/libs/cctz/src + contrib/libs/cxxsupp/libcxx-filesystem + contrib/libs/poco/Net + contrib/libs/poco/Util + contrib/libs/fmt + contrib/restricted/boost + contrib/restricted/cityhash-1.0.2 +) + +SRCS( + +) + +END() diff --git a/src/Disks/ya.make b/src/Disks/ya.make index f82cc4e2868..a64e5508ca8 100644 --- a/src/Disks/ya.make +++ b/src/Disks/ya.make @@ -7,6 +7,8 @@ PEERDIR( SRCS( createVolume.cpp + DiskCacheWrapper.cpp + DiskDecorator.cpp DiskFactory.cpp DiskLocal.cpp DiskMemory.cpp From f459640269fab608d2178ddcd660cd438df63f78 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Aug 2020 02:41:34 +0300 Subject: [PATCH 197/374] Fix build --- programs/server/CMakeLists.txt | 45 ++++++++++++++++++---------------- 1 file changed, 24 insertions(+), 21 deletions(-) diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index 81fb7b6d623..f3e2e19c4a1 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -26,29 +26,32 @@ clickhouse_program_add(server) install(FILES config.xml users.xml DESTINATION ${CLICKHOUSE_ETC_DIR}/clickhouse-server COMPONENT clickhouse) -# Embed default config files as a resource into the binary. -# This is needed for two purposes: -# 1. Allow to run the binary without download of any other files. -# 2. Allow to implement "sudo clickhouse install" tool. +# TODO We actually need this on Mac, FreeBSD and AArch64. +if (OS_LINUX AND ARCH_AMD64) + # Embed default config files as a resource into the binary. + # This is needed for two purposes: + # 1. Allow to run the binary without download of any other files. + # 2. Allow to implement "sudo clickhouse install" tool. -foreach(CONFIG_FILE config users embedded) - set(CONFIG_OBJ ${CONFIG_FILE}.o) - set(CONFIG_OBJS ${CONFIG_OBJS} ${CONFIG_OBJ}) + foreach(CONFIG_FILE config users embedded) + set(CONFIG_OBJ ${CONFIG_FILE}.o) + set(CONFIG_OBJS ${CONFIG_OBJS} ${CONFIG_OBJ}) - # https://stackoverflow.com/questions/14776463/compile-and-add-an-object-file-from-a-binary-with-cmake - add_custom_command(OUTPUT ${CONFIG_OBJ} - COMMAND cd ${CMAKE_CURRENT_SOURCE_DIR} && ${OBJCOPY_PATH} -I binary -O elf64-x86-64 -B i386 ${CONFIG_FILE}.xml ${CMAKE_CURRENT_BINARY_DIR}/${CONFIG_OBJ} - COMMAND ${OBJCOPY_PATH} --rename-section .data=.rodata,alloc,load,readonly,data,contents - ${CMAKE_CURRENT_BINARY_DIR}/${CONFIG_OBJ} ${CMAKE_CURRENT_BINARY_DIR}/${CONFIG_OBJ}) + # https://stackoverflow.com/questions/14776463/compile-and-add-an-object-file-from-a-binary-with-cmake + add_custom_command(OUTPUT ${CONFIG_OBJ} + COMMAND cd ${CMAKE_CURRENT_SOURCE_DIR} && ${OBJCOPY_PATH} -I binary -O elf64-x86-64 -B i386 ${CONFIG_FILE}.xml ${CMAKE_CURRENT_BINARY_DIR}/${CONFIG_OBJ} + COMMAND ${OBJCOPY_PATH} --rename-section .data=.rodata,alloc,load,readonly,data,contents + ${CMAKE_CURRENT_BINARY_DIR}/${CONFIG_OBJ} ${CMAKE_CURRENT_BINARY_DIR}/${CONFIG_OBJ}) - set_source_files_properties(${CONFIG_OBJ} PROPERTIES EXTERNAL_OBJECT true GENERATED true) -endforeach(CONFIG_FILE) + set_source_files_properties(${CONFIG_OBJ} PROPERTIES EXTERNAL_OBJECT true GENERATED true) + endforeach(CONFIG_FILE) -add_library(clickhouse_server_configs STATIC ${CONFIG_OBJS}) -set_target_properties(clickhouse_server_configs PROPERTIES LINKER_LANGUAGE C) + add_library(clickhouse_server_configs STATIC ${CONFIG_OBJS}) + set_target_properties(clickhouse_server_configs PROPERTIES LINKER_LANGUAGE C) -# whole-archive prevents symbols from being discarded for unknown reason -# CMake can shuffle each of target_link_libraries arguments with other -# libraries in linker command. To avoid this we hardcode whole-archive -# library into single string. -add_dependencies(clickhouse-server-lib clickhouse_server_configs) + # whole-archive prevents symbols from being discarded for unknown reason + # CMake can shuffle each of target_link_libraries arguments with other + # libraries in linker command. To avoid this we hardcode whole-archive + # library into single string. + add_dependencies(clickhouse-server-lib clickhouse_server_configs) +endif () From 2c5a2b2f43f4e43e8b4931b818bd20650cd700ed Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 8 Aug 2020 19:34:35 +0300 Subject: [PATCH 198/374] Add function isValidIdentifier(). --- src/Common/StringUtils/StringUtils.h | 5 +++++ src/IO/WriteHelpers.cpp | 18 +++--------------- 2 files changed, 8 insertions(+), 15 deletions(-) diff --git a/src/Common/StringUtils/StringUtils.h b/src/Common/StringUtils/StringUtils.h index 50573694b7a..a1e8fb79435 100644 --- a/src/Common/StringUtils/StringUtils.h +++ b/src/Common/StringUtils/StringUtils.h @@ -122,6 +122,11 @@ inline bool isPrintableASCII(char c) return uc >= 32 && uc <= 126; /// 127 is ASCII DEL. } +inline bool isValidIdentifier(const std::string_view & str) +{ + return !str.empty() && isValidIdentifierBegin(str[0]) && std::all_of(str.begin() + 1, str.end(), isWordCharASCII); +} + /// Works assuming isAlphaASCII. inline char toLowerIfAlphaASCII(char c) { diff --git a/src/IO/WriteHelpers.cpp b/src/IO/WriteHelpers.cpp index 9fe194a70c8..a0a2a45c791 100644 --- a/src/IO/WriteHelpers.cpp +++ b/src/IO/WriteHelpers.cpp @@ -68,22 +68,10 @@ void writeException(const Exception & e, WriteBuffer & buf, bool with_stack_trac template static inline void writeProbablyQuotedStringImpl(const StringRef & s, WriteBuffer & buf, F && write_quoted_string) { - if (!s.size || !isValidIdentifierBegin(s.data[0])) - { - write_quoted_string(s, buf); - } + if (isValidIdentifier(std::string_view{s})) + writeString(s, buf); else - { - const char * pos = s.data + 1; - const char * end = s.data + s.size; - for (; pos < end; ++pos) - if (!isWordCharASCII(*pos)) - break; - if (pos != end) - write_quoted_string(s, buf); - else - writeString(s, buf); - } + write_quoted_string(s, buf); } void writeProbablyBackQuotedString(const StringRef & s, WriteBuffer & buf) From 61c3735dfcafcebc76241563f84374f7180cb9e6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 8 Aug 2020 21:55:14 +0300 Subject: [PATCH 199/374] Support compound identifiers for custom settings. --- src/Parsers/ASTSetQuery.cpp | 22 ++++++++++++ src/Parsers/ASTSetQuery.h | 15 +------- src/Parsers/ASTSettingsProfileElement.cpp | 3 +- src/Parsers/ParserSetQuery.cpp | 2 +- src/Parsers/ParserSettingsProfileElement.cpp | 2 +- src/Parsers/formatSettingName.cpp | 36 ++++++++++++++++++++ src/Parsers/formatSettingName.h | 14 ++++++++ src/Parsers/ya.make | 2 ++ 8 files changed, 79 insertions(+), 17 deletions(-) create mode 100644 src/Parsers/ASTSetQuery.cpp create mode 100644 src/Parsers/formatSettingName.cpp create mode 100644 src/Parsers/formatSettingName.h diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp new file mode 100644 index 00000000000..fd5409e4bdd --- /dev/null +++ b/src/Parsers/ASTSetQuery.cpp @@ -0,0 +1,22 @@ +#include +#include + + +namespace DB +{ +void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const +{ + if (is_standalone) + format.ostr << (format.hilite ? hilite_keyword : "") << "SET " << (format.hilite ? hilite_none : ""); + + for (auto it = changes.begin(); it != changes.end(); ++it) + { + if (it != changes.begin()) + format.ostr << ", "; + + formatSettingName(it->name, format.ostr); + format.ostr << " = " << applyVisitor(FieldVisitorToString(), it->value); + } +} + +} diff --git a/src/Parsers/ASTSetQuery.h b/src/Parsers/ASTSetQuery.h index 19277d7f53b..e5c3e31b3c2 100644 --- a/src/Parsers/ASTSetQuery.h +++ b/src/Parsers/ASTSetQuery.h @@ -8,7 +8,6 @@ namespace DB { - /** SET query */ class ASTSetQuery : public IAST @@ -23,19 +22,7 @@ public: ASTPtr clone() const override { return std::make_shared(*this); } - void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override - { - if (is_standalone) - settings.ostr << (settings.hilite ? hilite_keyword : "") << "SET " << (settings.hilite ? hilite_none : ""); - - for (auto it = changes.begin(); it != changes.end(); ++it) - { - if (it != changes.begin()) - settings.ostr << ", "; - - settings.ostr << it->name << " = " << applyVisitor(FieldVisitorToString(), it->value); - } - } + void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override; }; } diff --git a/src/Parsers/ASTSettingsProfileElement.cpp b/src/Parsers/ASTSettingsProfileElement.cpp index a7955411347..c0fb2965a2d 100644 --- a/src/Parsers/ASTSettingsProfileElement.cpp +++ b/src/Parsers/ASTSettingsProfileElement.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -31,7 +32,7 @@ void ASTSettingsProfileElement::formatImpl(const FormatSettings & settings, Form return; } - settings.ostr << setting_name; + formatSettingName(setting_name, settings.ostr); if (!value.isNull()) { diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index 30d681cb126..b57f5799304 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -15,7 +15,7 @@ namespace DB /// Parse `name = value`. bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & pos, Expected & expected) { - ParserIdentifier name_p; + ParserCompoundIdentifier name_p; ParserLiteral value_p; ParserToken s_eq(TokenType::Equals); diff --git a/src/Parsers/ParserSettingsProfileElement.cpp b/src/Parsers/ParserSettingsProfileElement.cpp index 39e1f2d3594..f1e66a296e7 100644 --- a/src/Parsers/ParserSettingsProfileElement.cpp +++ b/src/Parsers/ParserSettingsProfileElement.cpp @@ -118,7 +118,7 @@ namespace return IParserBase::wrapParseImpl(pos, [&] { ASTPtr name_ast; - if (!ParserIdentifier{}.parse(pos, name_ast, expected)) + if (!ParserCompoundIdentifier{}.parse(pos, name_ast, expected)) return false; String res_setting_name = getIdentifierName(name_ast); diff --git a/src/Parsers/formatSettingName.cpp b/src/Parsers/formatSettingName.cpp new file mode 100644 index 00000000000..3f30142716e --- /dev/null +++ b/src/Parsers/formatSettingName.cpp @@ -0,0 +1,36 @@ +#include +#include +#include +#include + + +namespace DB +{ + +void formatSettingName(const String & setting_name, std::ostream & out) +{ + if (isValidIdentifier(setting_name)) + { + out << setting_name; + return; + } + + std::vector parts; + splitInto<'.'>(parts, setting_name); + bool all_parts_are_identifiers = std::all_of(parts.begin(), parts.end(), isValidIdentifier); + if (all_parts_are_identifiers && !parts.empty()) + { + bool need_dot = false; + for (const auto & part : parts) + { + if (std::exchange(need_dot, true)) + out << "."; + out << part; + } + return; + } + + out << backQuote(setting_name); +} + +} diff --git a/src/Parsers/formatSettingName.h b/src/Parsers/formatSettingName.h new file mode 100644 index 00000000000..40f14d95b4f --- /dev/null +++ b/src/Parsers/formatSettingName.h @@ -0,0 +1,14 @@ +#pragma once + +#include + + +namespace DB +{ + +/// Outputs built-in or custom setting's name. +/// The function is like backQuoteIfNeed() but didn't quote with backticks +/// if the name consists of identifiers joined with dots. +void formatSettingName(const String & setting_name, std::ostream & out); + +} diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make index ada5975bcf0..06b49f2b5ab 100644 --- a/src/Parsers/ya.make +++ b/src/Parsers/ya.make @@ -44,6 +44,7 @@ SRCS( ASTSampleRatio.cpp ASTSelectQuery.cpp ASTSelectWithUnionQuery.cpp + ASTSetQuery.cpp ASTSetRoleQuery.cpp ASTSettingsProfileElement.cpp ASTShowAccessEntitiesQuery.cpp @@ -61,6 +62,7 @@ SRCS( ExpressionElementParsers.cpp ExpressionListParsers.cpp formatAST.cpp + formatSettingName.cpp IAST.cpp iostream_debug_helpers.cpp IParserBase.cpp From 7efd495c7ea1c702a2953896b7699584c3057f80 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 8 Aug 2020 21:55:26 +0300 Subject: [PATCH 200/374] Add test. --- .../queries/0_stateless/01418_custom_settings.reference | 4 ++++ tests/queries/0_stateless/01418_custom_settings.sql | 9 +++++++++ 2 files changed, 13 insertions(+) diff --git a/tests/queries/0_stateless/01418_custom_settings.reference b/tests/queries/0_stateless/01418_custom_settings.reference index 1e5fd30305a..5f239591218 100644 --- a/tests/queries/0_stateless/01418_custom_settings.reference +++ b/tests/queries/0_stateless/01418_custom_settings.reference @@ -15,3 +15,7 @@ custom_b NULL custom_c UInt64_50000 custom_d Float64_1.11 0 UInt8 + +test String +custom_compound.identifier.v1 \'test\' +CREATE SETTINGS PROFILE s1_01418 SETTINGS custom_compound.identifier.v2 = 100 diff --git a/tests/queries/0_stateless/01418_custom_settings.sql b/tests/queries/0_stateless/01418_custom_settings.sql index 968ec22b538..1c87b143b79 100644 --- a/tests/queries/0_stateless/01418_custom_settings.sql +++ b/tests/queries/0_stateless/01418_custom_settings.sql @@ -23,3 +23,12 @@ SET custom_e = 0; SELECT getSetting('custom_e') as v, toTypeName(v); SET invalid_custom = 8; -- { serverError 115 } -- Setting is neither a builtin nor started with one of the registered prefixes for user-defined settings. + +SELECT ''; +SET custom_compound.identifier.v1 = 'test'; +SELECT getSetting('custom_compound.identifier.v1') as v, toTypeName(v); +SELECT name, value FROM system.settings WHERE name = 'custom_compound.identifier.v1'; + +CREATE SETTINGS PROFILE s1_01418 SETTINGS custom_compound.identifier.v2 = 100; +SHOW CREATE SETTINGS PROFILE s1_01418; +DROP SETTINGS PROFILE s1_01418; From 8d616444e4fa723e6cb5e70d2189cb005c6b5d9e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Aug 2020 02:54:38 +0300 Subject: [PATCH 201/374] Support for AArch64 --- CMakeLists.txt | 6 ++++++ contrib/cctz-cmake/CMakeLists.txt | 4 ++-- programs/server/CMakeLists.txt | 6 +++--- 3 files changed, 11 insertions(+), 5 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 35491fdcacd..6f77118e159 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -145,6 +145,12 @@ set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic") find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy") if (OBJCOPY_PATH) message(STATUS "Using objcopy: ${OBJCOPY_PATH}.") + + if (ARCH_AMD64) + set(OBJCOPY_ARCH_OPTIONS -O elf64-x86-64 -B i386) + elseif (ARCH_AARCH64) + set(OBJCOPY_ARCH_OPTIONS -O elf64-aarch64 -B aarch64) + endif () else () message(FATAL_ERROR "Cannot find objcopy.") endif () diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 7eb768b8de9..7ca28fc12c3 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -26,7 +26,7 @@ if (USE_INTERNAL_CCTZ) # Build a libray with embedded tzdata - if (OS_LINUX AND ARCH_AMD64) + if (OS_LINUX) set (TIMEZONES Africa/Abidjan @@ -602,7 +602,7 @@ if (USE_INTERNAL_CCTZ) # https://stackoverflow.com/questions/14776463/compile-and-add-an-object-file-from-a-binary-with-cmake add_custom_command(OUTPUT ${TZ_OBJ} - COMMAND cd ${TZDIR} && ${OBJCOPY_PATH} -I binary -O elf64-x86-64 -B i386 ${TIMEZONE} ${CMAKE_CURRENT_BINARY_DIR}/${TZ_OBJ} + COMMAND cd ${TZDIR} && ${OBJCOPY_PATH} -I binary ${OBJCOPY_ARCH_OPTIONS} ${TIMEZONE} ${CMAKE_CURRENT_BINARY_DIR}/${TZ_OBJ} COMMAND ${OBJCOPY_PATH} --rename-section .data=.rodata,alloc,load,readonly,data,contents ${CMAKE_CURRENT_BINARY_DIR}/${TZ_OBJ} ${CMAKE_CURRENT_BINARY_DIR}/${TZ_OBJ}) diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index f3e2e19c4a1..22e226272b4 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -26,8 +26,8 @@ clickhouse_program_add(server) install(FILES config.xml users.xml DESTINATION ${CLICKHOUSE_ETC_DIR}/clickhouse-server COMPONENT clickhouse) -# TODO We actually need this on Mac, FreeBSD and AArch64. -if (OS_LINUX AND ARCH_AMD64) +# TODO We actually need this on Mac, FreeBSD. +if (OS_LINUX) # Embed default config files as a resource into the binary. # This is needed for two purposes: # 1. Allow to run the binary without download of any other files. @@ -39,7 +39,7 @@ if (OS_LINUX AND ARCH_AMD64) # https://stackoverflow.com/questions/14776463/compile-and-add-an-object-file-from-a-binary-with-cmake add_custom_command(OUTPUT ${CONFIG_OBJ} - COMMAND cd ${CMAKE_CURRENT_SOURCE_DIR} && ${OBJCOPY_PATH} -I binary -O elf64-x86-64 -B i386 ${CONFIG_FILE}.xml ${CMAKE_CURRENT_BINARY_DIR}/${CONFIG_OBJ} + COMMAND cd ${CMAKE_CURRENT_SOURCE_DIR} && ${OBJCOPY_PATH} -I binary ${OBJCOPY_ARCH_OPTIONS} ${CONFIG_FILE}.xml ${CMAKE_CURRENT_BINARY_DIR}/${CONFIG_OBJ} COMMAND ${OBJCOPY_PATH} --rename-section .data=.rodata,alloc,load,readonly,data,contents ${CMAKE_CURRENT_BINARY_DIR}/${CONFIG_OBJ} ${CMAKE_CURRENT_BINARY_DIR}/${CONFIG_OBJ}) From 2280619ad0c45a4beff9d68481c3fbf1c55b3a41 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 9 Aug 2020 04:45:10 +0300 Subject: [PATCH 202/374] fix secondary indices corruption in compact parts --- .../MergeTree/MergeTreeDataPartWriterOnDisk.cpp | 5 +++++ .../01419_skip_index_compact_parts.reference | 1 + .../0_stateless/01419_skip_index_compact_parts.sql | 12 ++++++++++++ 3 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/01419_skip_index_compact_parts.reference create mode 100644 tests/queries/0_stateless/01419_skip_index_compact_parts.sql diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 6f4585b819d..ddc8cd178db 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -261,6 +261,11 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block { limit = current_index_offset; } + else if (skip_index_current_data_mark == index_granularity.getMarksCount()) + { + /// Case, when last granule was exceeded and no new granule was created. + limit = rows - prev_pos; + } else { limit = index_granularity.getMarkRows(skip_index_current_data_mark); diff --git a/tests/queries/0_stateless/01419_skip_index_compact_parts.reference b/tests/queries/0_stateless/01419_skip_index_compact_parts.reference new file mode 100644 index 00000000000..209e3ef4b62 --- /dev/null +++ b/tests/queries/0_stateless/01419_skip_index_compact_parts.reference @@ -0,0 +1 @@ +20 diff --git a/tests/queries/0_stateless/01419_skip_index_compact_parts.sql b/tests/queries/0_stateless/01419_skip_index_compact_parts.sql new file mode 100644 index 00000000000..8b03f28df44 --- /dev/null +++ b/tests/queries/0_stateless/01419_skip_index_compact_parts.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS index_compact; + +CREATE TABLE index_compact(a UInt32, b UInt32, index i1 b type minmax granularity 1) + ENGINE = MergeTree ORDER BY a + SETTINGS min_rows_for_wide_part = 1000, index_granularity = 128, merge_max_block_size = 100; + +INSERT INTO index_compact SELECT number, toString(number) FROM numbers(100); +INSERT INTO index_compact SELECT number, toString(number) FROM numbers(30); + +OPTIMIZE TABLE index_compact FINAL; + +SELECT count() FROM index_compact WHERE b < 10; From 73c8e2c8bd8911f2554ac4a8ef2a8cc7b51e3ef2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Aug 2020 05:13:56 +0300 Subject: [PATCH 203/374] Fix shadow --- programs/install/Install.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index b4f9173020d..4a76608b9d7 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -72,10 +72,10 @@ namespace fs = std::filesystem; auto executeScript(const std::string & command, bool throw_on_error = false) { auto sh = ShellCommand::execute(command); - WriteBufferFromFileDescriptor stdout(STDOUT_FILENO); - WriteBufferFromFileDescriptor stderr(STDERR_FILENO); - copyData(sh->out, stdout); - copyData(sh->err, stderr); + WriteBufferFromFileDescriptor wb_stdout(STDOUT_FILENO); + WriteBufferFromFileDescriptor wb_stderr(STDERR_FILENO); + copyData(sh->out, wb_stdout); + copyData(sh->err, wb_stderr); if (throw_on_error) { From aa3b4bbfe0517dd3a3efb1359bd56b05bcecaf7a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Aug 2020 05:16:55 +0300 Subject: [PATCH 204/374] Fix build --- programs/server/CMakeLists.txt | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index f3e2e19c4a1..fbb5eef446d 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -3,6 +3,10 @@ set(CLICKHOUSE_SERVER_SOURCES Server.cpp ) +if (OS_LINUX AND ARCH_AMD64) + set (LINK_CONFIG_LIB INTERFACE "-Wl,--whole-archive $ -Wl,--no-whole-archive") +endif () + set (CLICKHOUSE_SERVER_LINK PRIVATE clickhouse_aggregate_functions @@ -16,7 +20,7 @@ set (CLICKHOUSE_SERVER_LINK clickhouse_table_functions string_utils - INTERFACE "-Wl,--whole-archive $ -Wl,--no-whole-archive" + ${LINK_CONFIG_LIB} PUBLIC daemon From 760b455ea3684cb9e0c78e24d9e0dbe4baa58505 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 9 Aug 2020 10:53:58 +0800 Subject: [PATCH 205/374] More test --- .../0_stateless/01440_to_date_monotonicity.reference | 1 + tests/queries/0_stateless/01440_to_date_monotonicity.sql | 7 +++++++ 2 files changed, 8 insertions(+) diff --git a/tests/queries/0_stateless/01440_to_date_monotonicity.reference b/tests/queries/0_stateless/01440_to_date_monotonicity.reference index 30b841d4487..96732e5996c 100644 --- a/tests/queries/0_stateless/01440_to_date_monotonicity.reference +++ b/tests/queries/0_stateless/01440_to_date_monotonicity.reference @@ -1,3 +1,4 @@ 0 1970-01-01 2106-02-07 1970-04-11 1970-01-01 2106-02-07 1970-01-01 03:00:00 2106-02-07 09:28:15 1970-01-01 03:16:40 +2000-01-01 13:12:12 diff --git a/tests/queries/0_stateless/01440_to_date_monotonicity.sql b/tests/queries/0_stateless/01440_to_date_monotonicity.sql index 0355d1fec30..e48911e954c 100644 --- a/tests/queries/0_stateless/01440_to_date_monotonicity.sql +++ b/tests/queries/0_stateless/01440_to_date_monotonicity.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS tdm; +DROP TABLE IF EXISTS tdm2; CREATE TABLE tdm (x DateTime) ENGINE = MergeTree ORDER BY x SETTINGS write_final_mark = 0; INSERT INTO tdm VALUES (now()); SELECT count(x) FROM tdm WHERE toDate(x) < today() SETTINGS max_rows_to_read = 1; @@ -6,4 +7,10 @@ SELECT count(x) FROM tdm WHERE toDate(x) < today() SETTINGS max_rows_to_read = 1 SELECT toDate(-1), toDate(10000000000000), toDate(100), toDate(65536), toDate(65535); SELECT toDateTime(-1), toDateTime(10000000000000), toDateTime(1000); +CREATE TABLE tdm2 (timestamp UInt32) ENGINE = MergeTree ORDER BY timestamp SETTINGS index_granularity = 1; +INSERT INTO tdm2 VALUES (toUnixTimestamp('2000-01-01 13:12:12')), (toUnixTimestamp('2000-01-01 14:12:12')), (toUnixTimestamp('2000-01-01 15:12:12')); +SET max_rows_to_read = 1; +SELECT toDateTime(timestamp) FROM tdm2 WHERE toHour(toDateTime(timestamp)) = 13; + DROP TABLE tdm; +DROP TABLE tdm2; From 933f287db4ca6004b3247411b245b5af6bf24351 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 9 Aug 2020 20:10:30 +0800 Subject: [PATCH 206/374] ISSUES-4006 resolve review comment & support alter renmae command --- .../MySQL/MaterializeMySQLSyncThread.cpp | 20 ++---- .../MySQL/MaterializeMySQLSyncThread.h | 2 - .../MySQL/InterpretersMySQLDDLQuery.cpp | 64 +++++++++++++------ .../MySQL/InterpretersMySQLDDLQuery.h | 14 ++-- .../MySQL/tests/gtest_create_rewritten.cpp | 4 +- src/Parsers/MySQL/ASTAlterCommand.cpp | 9 ++- src/Parsers/MySQL/ASTAlterCommand.h | 4 +- .../tests/gtest_alter_command_parser.cpp | 2 +- .../materialize_with_ddl.py | 27 ++++++++ .../test_materialize_mysql_database/test.py | 2 + 10 files changed, 96 insertions(+), 52 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 747adc244f4..b53d84189ab 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -168,8 +168,6 @@ void MaterializeMySQLSyncThread::synchronization(const String & mysql_version) BinlogEventPtr binlog_event = client.readOneBinlogEvent(std::max(UInt64(1), max_flush_time - watch.elapsedMilliseconds())); { - std::unique_lock lock(sync_mutex); - if (binlog_event) onEvent(buffers, binlog_event, *metadata); @@ -198,12 +196,7 @@ void MaterializeMySQLSyncThread::stopSynchronization() { if (!sync_quit && background_thread_pool) { - { - sync_quit = true; - std::lock_guard lock(sync_mutex); - } - - sync_cond.notify_one(); + sync_quit = true; background_thread_pool->join(); } } @@ -277,8 +270,6 @@ static inline UInt32 randomNumber() std::optional MaterializeMySQLSyncThread::prepareSynchronized(const String & mysql_version) { - std::unique_lock lock(sync_mutex); - bool opened_transaction = false; mysqlxx::PoolWithFailover::Entry connection; @@ -286,11 +277,6 @@ std::optional MaterializeMySQLSyncThread::prepareSynchroniz { try { - LOG_DEBUG(log, "Checking database status."); - while (!isCancelled() && !DatabaseCatalog::instance().isDatabaseExist(database_name)) - sync_cond.wait_for(lock, std::chrono::seconds(1)); - LOG_DEBUG(log, "Database status is OK."); - connection = pool.get(); opened_transaction = false; @@ -325,7 +311,7 @@ std::optional MaterializeMySQLSyncThread::prepareSynchroniz { throw; } - catch (mysqlxx::Exception &) + catch (const mysqlxx::ConnectionFailed &) { /// Avoid busy loop when MySQL is not available. sleepForMilliseconds(settings->max_wait_time_when_mysql_unavailable); @@ -586,6 +572,8 @@ void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr { tryLogCurrentException(log); + /// If some DDL query was not successfully parsed and executed + /// Then replication may fail on next binlog events anyway if (exception.code() != ErrorCodes::SYNTAX_ERROR) throw; } diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index 9c454de2a3f..599efc2fb42 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -98,9 +98,7 @@ private: void onEvent(Buffers & buffers, const MySQLReplication::BinlogEventPtr & event, MaterializeMetadata & metadata); - std::mutex sync_mutex; std::atomic sync_quit{false}; - std::condition_variable sync_cond; std::unique_ptr background_thread_pool; }; diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 95bd8c44d30..31eb3324b94 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -321,7 +321,7 @@ void InterpreterCreateImpl::validate(const InterpreterCreateImpl::TQuery & creat throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); } -ASTPtr InterpreterCreateImpl::getRewrittenQuery( +ASTs InterpreterCreateImpl::getRewrittenQueries( const TQuery & create_query, const Context & context, const String & mapped_to_database, const String & mysql_database) { auto rewritten_query = std::make_shared(); @@ -364,14 +364,14 @@ ASTPtr InterpreterCreateImpl::getRewrittenQuery( rewritten_query->set(rewritten_query->storage, storage); rewritten_query->set(rewritten_query->columns_list, columns); - return rewritten_query; + return ASTs{rewritten_query}; } void InterpreterDropImpl::validate(const InterpreterDropImpl::TQuery & /*query*/, const Context & /*context*/) { } -ASTPtr InterpreterDropImpl::getRewrittenQuery( +ASTs InterpreterDropImpl::getRewrittenQueries( const InterpreterDropImpl::TQuery & drop_query, const Context & context, const String & mapped_to_database, const String & mysql_database) { const auto & database_name = resolveDatabase(drop_query.database, mysql_database, mapped_to_database, context); @@ -382,7 +382,7 @@ ASTPtr InterpreterDropImpl::getRewrittenQuery( ASTPtr rewritten_query = drop_query.clone(); rewritten_query->as()->database = mapped_to_database; - return rewritten_query; + return ASTs{rewritten_query}; } void InterpreterRenameImpl::validate(const InterpreterRenameImpl::TQuery & rename_query, const Context & /*context*/) @@ -391,7 +391,7 @@ void InterpreterRenameImpl::validate(const InterpreterRenameImpl::TQuery & renam throw Exception("Cannot execute exchange for external ddl query.", ErrorCodes::NOT_IMPLEMENTED); } -ASTPtr InterpreterRenameImpl::getRewrittenQuery( +ASTs InterpreterRenameImpl::getRewrittenQueries( const InterpreterRenameImpl::TQuery & rename_query, const Context & context, const String & mapped_to_database, const String & mysql_database) { ASTRenameQuery::Elements elements; @@ -414,27 +414,28 @@ ASTPtr InterpreterRenameImpl::getRewrittenQuery( } if (elements.empty()) - return ASTPtr{}; + return ASTs{}; auto rewritten_query = std::make_shared(); rewritten_query->elements = elements; - return rewritten_query; + return ASTs{rewritten_query}; } void InterpreterAlterImpl::validate(const InterpreterAlterImpl::TQuery & /*query*/, const Context & /*context*/) { } -ASTPtr InterpreterAlterImpl::getRewrittenQuery( +ASTs InterpreterAlterImpl::getRewrittenQueries( const InterpreterAlterImpl::TQuery & alter_query, const Context & context, const String & mapped_to_database, const String & mysql_database) { if (resolveDatabase(alter_query.database, mysql_database, mapped_to_database, context) != mapped_to_database) return {}; - auto rewritten_query = std::make_shared(); - rewritten_query->database = mapped_to_database; - rewritten_query->table = alter_query.table; - rewritten_query->set(rewritten_query->command_list, std::make_shared()); + auto rewritten_alter_query = std::make_shared(); + auto rewritten_rename_query = std::make_shared(); + rewritten_alter_query->database = mapped_to_database; + rewritten_alter_query->table = alter_query.table; + rewritten_alter_query->set(rewritten_alter_query->command_list, std::make_shared()); String default_after_column; for (const auto & command_query : alter_query.command_list->children) @@ -498,7 +499,7 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( } rewritten_command->children.push_back(rewritten_command->col_decl); - rewritten_query->command_list->add(rewritten_command); + rewritten_alter_query->command_list->add(rewritten_command); } } else if (alter_command->type == MySQLParser::ASTAlterCommand::DROP_COLUMN) @@ -506,7 +507,7 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( auto rewritten_command = std::make_shared(); rewritten_command->type = ASTAlterCommand::DROP_COLUMN; rewritten_command->column = std::make_shared(alter_command->column_name); - rewritten_query->command_list->add(rewritten_command); + rewritten_alter_query->command_list->add(rewritten_command); } else if (alter_command->type == MySQLParser::ASTAlterCommand::RENAME_COLUMN) { @@ -517,7 +518,7 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( rewritten_command->type = ASTAlterCommand::RENAME_COLUMN; rewritten_command->column = std::make_shared(alter_command->old_name); rewritten_command->rename_to = std::make_shared(alter_command->column_name); - rewritten_query->command_list->add(rewritten_command); + rewritten_alter_query->command_list->add(rewritten_command); } } else if (alter_command->type == MySQLParser::ASTAlterCommand::MODIFY_COLUMN) @@ -546,7 +547,7 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( rewritten_command->children.push_back(rewritten_command->column); } - rewritten_query->command_list->add(rewritten_command); + rewritten_alter_query->command_list->add(rewritten_command); } if (!alter_command->old_name.empty() && alter_command->old_name != new_column_name) @@ -555,15 +556,38 @@ ASTPtr InterpreterAlterImpl::getRewrittenQuery( rewritten_command->type = ASTAlterCommand::RENAME_COLUMN; rewritten_command->column = std::make_shared(alter_command->old_name); rewritten_command->rename_to = std::make_shared(new_column_name); - rewritten_query->command_list->add(rewritten_command); + rewritten_alter_query->command_list->add(rewritten_command); } } + else if (alter_command->type == MySQLParser::ASTAlterCommand::RENAME_TABLE) + { + const auto & to_database = resolveDatabase(alter_command->new_database_name, mysql_database, mapped_to_database, context); + + if (to_database != mapped_to_database) + throw Exception("Cannot rename with other database for external ddl query.", ErrorCodes::NOT_IMPLEMENTED); + + /// For ALTER TABLE table_name RENAME TO new_table_name_1, RENAME TO new_table_name_2; + /// We just need to generate RENAME TABLE table_name TO new_table_name_2; + if (rewritten_rename_query->elements.empty()) + rewritten_rename_query->elements.push_back(ASTRenameQuery::Element()); + + rewritten_rename_query->elements.back().from.database = mapped_to_database; + rewritten_rename_query->elements.back().from.table = alter_query.table; + rewritten_rename_query->elements.back().to.database = mapped_to_database; + rewritten_rename_query->elements.back().to.table = alter_command->new_table_name; + } } - if (rewritten_query->command_list->commands.empty()) - return {}; + ASTs rewritten_queries; - return rewritten_query; + /// Order is very important. We always execute alter first and then execute rename + if (!rewritten_alter_query->command_list->commands.empty()) + rewritten_queries.push_back(rewritten_alter_query); + + if (!rewritten_rename_query->elements.empty()) + rewritten_queries.push_back(rewritten_rename_query); + + return rewritten_queries; } } diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h index 8fc01c78d9a..497a661cc7f 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h @@ -21,7 +21,7 @@ struct InterpreterDropImpl static void validate(const TQuery & query, const Context & context); - static ASTPtr getRewrittenQuery(const TQuery & drop_query, const Context & context, const String & mapped_to_database, const String & mysql_database); + static ASTs getRewrittenQueries(const TQuery & drop_query, const Context & context, const String & mapped_to_database, const String & mysql_database); }; struct InterpreterAlterImpl @@ -30,7 +30,7 @@ struct InterpreterAlterImpl static void validate(const TQuery & query, const Context & context); - static ASTPtr getRewrittenQuery(const TQuery & alter_query, const Context & context, const String & mapped_to_database, const String & mysql_database); + static ASTs getRewrittenQueries(const TQuery & alter_query, const Context & context, const String & mapped_to_database, const String & mysql_database); }; struct InterpreterRenameImpl @@ -39,7 +39,7 @@ struct InterpreterRenameImpl static void validate(const TQuery & query, const Context & context); - static ASTPtr getRewrittenQuery(const TQuery & rename_query, const Context & context, const String & mapped_to_database, const String & mysql_database); + static ASTs getRewrittenQueries(const TQuery & rename_query, const Context & context, const String & mapped_to_database, const String & mysql_database); }; struct InterpreterCreateImpl @@ -48,7 +48,7 @@ struct InterpreterCreateImpl static void validate(const TQuery & query, const Context & context); - static ASTPtr getRewrittenQuery(const TQuery & create_query, const Context & context, const String & mapped_to_database, const String & mysql_database); + static ASTs getRewrittenQueries(const TQuery & create_query, const Context & context, const String & mapped_to_database, const String & mysql_database); }; template @@ -65,10 +65,10 @@ public: const typename InterpreterImpl::TQuery & query = query_ptr->as(); InterpreterImpl::validate(query, context); - ASTPtr rewritten_query = InterpreterImpl::getRewrittenQuery(query, context, mapped_to_database, mysql_database); + ASTs rewritten_queries = InterpreterImpl::getRewrittenQueries(query, context, mapped_to_database, mysql_database); - if (rewritten_query) - return executeQuery("/* Rewritten MySQL DDL Query */ " + queryToString(rewritten_query), context, true); + for (const auto & rewritten_query : rewritten_queries) + executeQuery("/* Rewritten MySQL DDL Query */ " + queryToString(rewritten_query), context, true); return BlockIO{}; } diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index 3782d20bd3d..c8aa2d59b3f 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -21,9 +21,9 @@ static inline ASTPtr tryRewrittenCreateQuery(const String & query, const Context ParserExternalDDLQuery external_ddl_parser; ASTPtr ast = parseQuery(external_ddl_parser, query, 0, 0); - return MySQLInterpreter::InterpreterCreateImpl::getRewrittenQuery( + return MySQLInterpreter::InterpreterCreateImpl::getRewrittenQueries( *ast->as()->external_ddl->as(), - context, "test_database", "test_database"); + context, "test_database", "test_database")[0]; } TEST(MySQLCreateRewritten, RewrittenQueryWithPrimaryKey) diff --git a/src/Parsers/MySQL/ASTAlterCommand.cpp b/src/Parsers/MySQL/ASTAlterCommand.cpp index c95aacb4e28..b6f2b925de0 100644 --- a/src/Parsers/MySQL/ASTAlterCommand.cpp +++ b/src/Parsers/MySQL/ASTAlterCommand.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -241,11 +242,13 @@ static inline bool parseRenameCommand(IParser::Pos & pos, ASTPtr & node, Expecte } else if (ParserKeyword("TO").ignore(pos, expected) || ParserKeyword("AS").ignore(pos, expected)) { - if (!identifier_p.parse(pos, new_name, expected)) + if (!ParserCompoundIdentifier(false).parse(pos, new_name, expected)) return false; - alter_command->type = ASTAlterCommand::RENAME_FOREIGN; - alter_command->index_name = getIdentifierName(new_name); + StorageID new_table_id = getTableIdentifier(new_name); + alter_command->type = ASTAlterCommand::RENAME_TABLE; + alter_command->new_table_name = new_table_id.table_name; + alter_command->new_database_name = new_table_id.database_name; } else if (ParserKeyword("INDEX").ignore(pos, expected) || ParserKeyword("KEY").ignore(pos, expected)) { diff --git a/src/Parsers/MySQL/ASTAlterCommand.h b/src/Parsers/MySQL/ASTAlterCommand.h index fa8fee83fd3..933a9700c70 100644 --- a/src/Parsers/MySQL/ASTAlterCommand.h +++ b/src/Parsers/MySQL/ASTAlterCommand.h @@ -31,7 +31,7 @@ public: RENAME_INDEX, RENAME_COLUMN, - RENAME_FOREIGN, + RENAME_TABLE, MODIFY_CHECK, MODIFY_COLUMN, @@ -66,6 +66,8 @@ public: String index_name; String column_name; String constraint_name; + String new_database_name; + String new_table_name; IAST * properties = nullptr; diff --git a/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp b/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp index 5534001241c..8ac781b533a 100644 --- a/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp +++ b/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp @@ -160,7 +160,7 @@ TEST(ParserAlterCommand, RenameAlterCommand) EXPECT_EQ(ast->as()->index_name, "new_index_name"); ast = tryParserQuery(alter_p, "RENAME TO new_table_name"); - EXPECT_EQ(ast->as()->type, ASTAlterCommand::RENAME_FOREIGN); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::RENAME_TABLE); EXPECT_EQ(ast->as()->index_name, "new_table_name"); } diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index ec98b4109ce..d2516c03572 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -292,6 +292,33 @@ def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_n mysql_node.query("DROP DATABASE test_database") clickhouse_node.query("DROP DATABASE test_database") + # TODO: need ClickHouse support ALTER TABLE table_name ADD COLUMN column_name, RENAME COLUMN column_name TO new_column_name; # def test_mysql_alter_change_column_for_materialize_mysql_database(started_cluster): # pass + +def alter_rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") + + mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column, RENAME TO test_database.test_table_2, RENAME TO test_table_3") + + # create mapping + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) + + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_3\n") + check_query(clickhouse_node, "DESC test_database.test_table_3 FORMAT TSV", "id\tInt32\t\t\t\t\t\n") + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_3\n") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\ndrop_column\tInt32\t\t\t\t\t\n") + mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column, RENAME TO test_table_2, RENAME TO test_database.test_table_4") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_3\ntest_table_4\n") + check_query(clickhouse_node, "DESC test_database.test_table_4 FORMAT TSV", "id\tInt32\t\t\t\t\t\n") + + mysql_node.query("INSERT INTO test_database.test_table_4 VALUES(1), (2), (3), (4), (5)") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_4 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") + + mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 6100d0bba37..26f8b43da0e 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -103,6 +103,7 @@ def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_ materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") # mysql 5.7 cannot support alter rename column # materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") @@ -112,6 +113,7 @@ def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_ materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") From fe566978930a12ed5b89c4467bb5acb0a3acc0ca Mon Sep 17 00:00:00 2001 From: Konstantin Podshumok Date: Sun, 9 Aug 2020 06:42:09 +0300 Subject: [PATCH 207/374] cmake: Add option to fail configuration instead of auto-reconfiguration squashed: find/msgpack.cmake - merge collision fix, trivial Signed-off-by: Konstantin Podshumok --- CMakeLists.txt | 14 +++- cmake/analysis.cmake | 4 +- cmake/contrib_finder.cmake | 4 +- cmake/find/amqpcpp.cmake | 25 +++--- cmake/find/avro.cmake | 18 +++-- cmake/find/base64.cmake | 26 +++--- cmake/find/brotli.cmake | 14 +++- cmake/find/capnp.cmake | 26 ++++-- cmake/find/cassandra.cmake | 40 +++++----- cmake/find/ccache.cmake | 10 ++- cmake/find/cxx.cmake | 71 +++++++++++----- cmake/find/fastops.cmake | 27 ++++--- cmake/find/gperf.cmake | 11 ++- cmake/find/grpc.cmake | 50 +++++++----- cmake/find/gtest.cmake | 17 +++- cmake/find/h3.cmake | 38 +++++---- cmake/find/hdfs3.cmake | 22 +++-- cmake/find/icu.cmake | 13 ++- cmake/find/ldap.cmake | 151 +++++++++++++++++++---------------- cmake/find/libgsasl.cmake | 17 ++-- cmake/find/libxml2.cmake | 5 ++ cmake/find/llvm.cmake | 139 +++++++++++++++++--------------- cmake/find/ltdl.cmake | 2 +- cmake/find/msgpack.cmake | 32 +++++--- cmake/find/mysqlclient.cmake | 93 ++++++++++++--------- cmake/find/opencl.cmake | 10 ++- cmake/find/orc.cmake | 19 ++++- cmake/find/parquet.cmake | 40 +++++++--- cmake/find/protobuf.cmake | 13 ++- cmake/find/rapidjson.cmake | 7 ++ cmake/find/rdkafka.cmake | 51 +++++++++--- cmake/find/re2.cmake | 7 +- cmake/find/s3.cmake | 51 +++++++----- cmake/find/sentry.cmake | 5 ++ cmake/find/simdjson.cmake | 7 +- cmake/find/snappy.cmake | 26 +++--- cmake/find/sparsehash.cmake | 3 + cmake/find/ssl.cmake | 15 +++- cmake/find/stats.cmake | 4 + cmake/find/termcap.cmake | 3 + cmake/find/zlib.cmake | 3 + cmake/find/zstd.cmake | 6 +- cmake/tools.cmake | 22 +++-- 43 files changed, 744 insertions(+), 417 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 35491fdcacd..06d935140bf 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -28,6 +28,14 @@ endforeach() project(ClickHouse) +option(FAIL_ON_UNSUPPORTED_OPTIONS_COMBINATION + "Stop/Fail CMake configuration if some ENABLE_XXX option is defined (either ON or OFF) but is not possible to satisfy" OFF) +if(FAIL_ON_UNSUPPORTED_OPTIONS_COMBINATION) + set(RECONFIGURE_MESSAGE_LEVEL FATAL_ERROR) +else() + set(RECONFIGURE_MESSAGE_LEVEL STATUS) +endif() + include (cmake/arch.cmake) include (cmake/target.cmake) include (cmake/tools.cmake) @@ -57,7 +65,7 @@ if(ENABLE_IPO) message(STATUS "IPO/LTO is supported, enabling") set(CMAKE_INTERPROCEDURAL_OPTIMIZATION TRUE) else() - message(STATUS "IPO/LTO is not supported: <${IPO_NOT_SUPPORTED}>") + message (${RECONFIGURE_MESSAGE_LEVEL} "IPO/LTO is not supported: <${IPO_NOT_SUPPORTED}>") endif() else() message(STATUS "IPO/LTO not enabled.") @@ -133,6 +141,8 @@ option (ENABLE_TESTS "Enables tests" ON) if (OS_LINUX AND NOT UNBUNDLED AND MAKE_STATIC_LIBRARIES AND NOT SPLIT_SHARED_LIBRARIES AND CMAKE_VERSION VERSION_GREATER "3.9.0") option (GLIBC_COMPATIBILITY "Set to TRUE to enable compatibility with older glibc libraries. Only for x86_64, Linux. Implies ENABLE_FASTMEMCPY." ON) +elseif(GLIBC_COMPATIBILITY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Glibc compatibility cannot be enabled in current configuration") endif () if (NOT CMAKE_VERSION VERSION_GREATER "3.9.0") @@ -242,6 +252,8 @@ if (COMPILER_CLANG) set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -flto=thin") set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -flto=thin") set (CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO "${CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO} -flto=thin") + elseif (ENABLE_THINLTO) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot enable ThinLTO") endif () # Always prefer llvm tools when using clang. For instance, we cannot use GNU ar when llvm LTO is enabled diff --git a/cmake/analysis.cmake b/cmake/analysis.cmake index 287c36a8de7..daaa730ac4b 100644 --- a/cmake/analysis.cmake +++ b/cmake/analysis.cmake @@ -12,7 +12,9 @@ if (ENABLE_CLANG_TIDY) set (USE_CLANG_TIDY 1) # The variable CMAKE_CXX_CLANG_TIDY will be set inside src and base directories with non third-party code. # set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") + elseif (FAIL_ON_UNSUPPORTED_OPTIONS_COMBINATION) + message(FATAL_ERROR "clang-tidy is not found") else () - message(STATUS "clang-tidy is not found. This is normal - the tool is used only for static code analysis and not essential for build.") + message(STATUS "clang-tidy is not found. This is normal - the tool is only used for static code analysis and isn't essential for the build.") endif () endif () diff --git a/cmake/contrib_finder.cmake b/cmake/contrib_finder.cmake index f9bf3ce2837..64c6d5f5c0a 100644 --- a/cmake/contrib_finder.cmake +++ b/cmake/contrib_finder.cmake @@ -8,6 +8,9 @@ macro(find_contrib_lib LIB_NAME) if (NOT USE_INTERNAL_${LIB_NAME_UC}_LIBRARY) find_package ("${LIB_NAME}") + if (NOT ${LIB_NAME_UC}_FOUND) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use system ${LIB_NAME}") + endif() endif () if (NOT ${LIB_NAME_UC}_FOUND) @@ -17,5 +20,4 @@ macro(find_contrib_lib LIB_NAME) endif () message (STATUS "Using ${LIB_NAME}: ${${LIB_NAME_UC}_INCLUDE_DIR} : ${${LIB_NAME_UC}_LIBRARIES}") - endmacro() diff --git a/cmake/find/amqpcpp.cmake b/cmake/find/amqpcpp.cmake index b3e193c72ff..4191dce26bb 100644 --- a/cmake/find/amqpcpp.cmake +++ b/cmake/find/amqpcpp.cmake @@ -1,21 +1,22 @@ option(ENABLE_AMQPCPP "Enalbe AMQP-CPP" ${ENABLE_LIBRARIES}) +if (NOT ENABLE_AMQPCPP) + return() +endif() + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP/CMakeLists.txt") message (WARNING "submodule contrib/AMQP-CPP is missing. to fix try run: \n git submodule update --init --recursive") - set (ENABLE_AMQPCPP 0) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal AMQP-CPP library") + set (USE_AMQPCPP 0) + return() endif () -if (ENABLE_AMQPCPP) +set (USE_AMQPCPP 1) +set (AMQPCPP_LIBRARY AMQP-CPP) - set (USE_AMQPCPP 1) - set (AMQPCPP_LIBRARY AMQP-CPP) - - set (AMQPCPP_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP/include") - - list (APPEND AMQPCPP_INCLUDE_DIR - "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP/include" - "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP") - -endif() +set (AMQPCPP_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP/include") +list (APPEND AMQPCPP_INCLUDE_DIR + "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP/include" + "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP") message (STATUS "Using AMQP-CPP=${USE_AMQPCPP}: ${AMQPCPP_INCLUDE_DIR} : ${AMQPCPP_LIBRARY}") diff --git a/cmake/find/avro.cmake b/cmake/find/avro.cmake index cdb3fc84d3d..a07e6353212 100644 --- a/cmake/find/avro.cmake +++ b/cmake/find/avro.cmake @@ -1,28 +1,34 @@ option (ENABLE_AVRO "Enable Avro" ${ENABLE_LIBRARIES}) -if (ENABLE_AVRO) +if (NOT ENABLE_AVRO) + if (USE_INTERNAL_AVRO_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal avro library with ENABLE_AVRO=OFF") + endif() + return() +endif() option (USE_INTERNAL_AVRO_LIBRARY "Set to FALSE to use system avro library instead of bundled" ${NOT_UNBUNDLED}) -if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/avro/lang/c++/CMakeLists.txt") - if(USE_INTERNAL_AVRO_LIBRARY) +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/avro/lang/c++/CMakeLists.txt") + if (USE_INTERNAL_AVRO_LIBRARY) message(WARNING "submodule contrib/avro is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot find internal avro") + set(USE_INTERNAL_AVRO_LIBRARY 0) endif() set(MISSING_INTERNAL_AVRO_LIBRARY 1) - set(USE_INTERNAL_AVRO_LIBRARY 0) endif() if (NOT USE_INTERNAL_AVRO_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Using system avro library is not supported yet") elseif(NOT MISSING_INTERNAL_AVRO_LIBRARY) include(cmake/find/snappy.cmake) set(AVROCPP_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/avro/lang/c++/include") set(AVROCPP_LIBRARY avrocpp) + set(USE_INTERNAL_AVRO_LIBRARY 1) endif () if (AVROCPP_LIBRARY AND AVROCPP_INCLUDE_DIR) set(USE_AVRO 1) endif() -endif() - message (STATUS "Using avro=${USE_AVRO}: ${AVROCPP_INCLUDE_DIR} : ${AVROCPP_LIBRARY}") diff --git a/cmake/find/base64.cmake b/cmake/find/base64.cmake index f72397597d7..7427baf9cad 100644 --- a/cmake/find/base64.cmake +++ b/cmake/find/base64.cmake @@ -1,17 +1,21 @@ +option (ENABLE_BASE64 "Enable base64" ${ENABLE_LIBRARIES}) + +if (NOT ENABLE_BASE64) + return() +endif() + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/base64/LICENSE") set (MISSING_INTERNAL_BASE64_LIBRARY 1) message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init --recursive") endif () -if (NOT MISSING_INTERNAL_BASE64_LIBRARY) - option (ENABLE_BASE64 "Enable base64" ${ENABLE_LIBRARIES}) -endif () +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/base64") + message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init --recursive") +else() + set (BASE64_LIBRARY base64) + set (USE_BASE64 1) +endif() -if (ENABLE_BASE64) - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/base64") - message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init --recursive") - else() - set (BASE64_LIBRARY base64) - set (USE_BASE64 1) - endif() -endif () +if (NOT USE_BASE64) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot enable base64") +endif() diff --git a/cmake/find/brotli.cmake b/cmake/find/brotli.cmake index 5453cf15236..eca3df0d2c8 100644 --- a/cmake/find/brotli.cmake +++ b/cmake/find/brotli.cmake @@ -1,12 +1,18 @@ option (ENABLE_BROTLI "Enable brotli" ${ENABLE_LIBRARIES}) -if (ENABLE_BROTLI) +if (NOT ENABLE_BROTLI) + if (USE_INTERNAL_BROTLI_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal brotly library with ENABLE_BROTLI=OFF") + endif() + return() +endif() option (USE_INTERNAL_BROTLI_LIBRARY "Set to FALSE to use system libbrotli library instead of bundled" ${NOT_UNBUNDLED}) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/brotli/c/include/brotli/decode.h") if (USE_INTERNAL_BROTLI_LIBRARY) message (WARNING "submodule contrib/brotli is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot find internal brotli") set (USE_INTERNAL_BROTLI_LIBRARY 0) endif () set (MISSING_INTERNAL_BROTLI_LIBRARY 1) @@ -19,18 +25,18 @@ if(NOT USE_INTERNAL_BROTLI_LIBRARY) find_path(BROTLI_INCLUDE_DIR NAMES brotli/decode.h brotli/encode.h brotli/port.h brotli/types.h PATHS ${BROTLI_INCLUDE_PATHS}) if(BROTLI_LIBRARY_DEC AND BROTLI_LIBRARY_ENC AND BROTLI_LIBRARY_COMMON) set(BROTLI_LIBRARY ${BROTLI_LIBRARY_DEC} ${BROTLI_LIBRARY_ENC} ${BROTLI_LIBRARY_COMMON}) + else() + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use system brotli") endif() endif() if (BROTLI_LIBRARY AND BROTLI_INCLUDE_DIR) set (USE_BROTLI 1) elseif (NOT MISSING_INTERNAL_BROTLI_LIBRARY) - set (BROTLI_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/brotli/c/include) + set (BROTLI_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/brotli/c/include") set (USE_INTERNAL_BROTLI_LIBRARY 1) set (BROTLI_LIBRARY brotli) set (USE_BROTLI 1) endif () -endif() - message (STATUS "Using brotli=${USE_BROTLI}: ${BROTLI_INCLUDE_DIR} : ${BROTLI_LIBRARY}") diff --git a/cmake/find/capnp.cmake b/cmake/find/capnp.cmake index 0620a66808b..ee4735bd175 100644 --- a/cmake/find/capnp.cmake +++ b/cmake/find/capnp.cmake @@ -1,15 +1,21 @@ option (ENABLE_CAPNP "Enable Cap'n Proto" ${ENABLE_LIBRARIES}) -if (ENABLE_CAPNP) +if (NOT ENABLE_CAPNP) + if (USE_INTERNAL_CAPNP_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal capnproto library with ENABLE_CAPNP=OFF") + endif() + return() +endif() option (USE_INTERNAL_CAPNP_LIBRARY "Set to FALSE to use system capnproto library instead of bundled" ${NOT_UNBUNDLED}) if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/capnproto/CMakeLists.txt") if(USE_INTERNAL_CAPNP_LIBRARY) message(WARNING "submodule contrib/capnproto is missing. to fix try run: \n git submodule update --init --recursive") + message(${RECONFIGURE_MESSAGE_LEVEL} "cannot find internal capnproto") + set(USE_INTERNAL_CAPNP_LIBRARY 0) endif() set(MISSING_INTERNAL_CAPNP_LIBRARY 1) - set(USE_INTERNAL_CAPNP_LIBRARY 0) endif() # FIXME: refactor to use `add_library(… IMPORTED)` if possible. @@ -18,17 +24,21 @@ if (NOT USE_INTERNAL_CAPNP_LIBRARY) find_library (CAPNP capnp) find_library (CAPNPC capnpc) - set (CAPNP_LIBRARIES ${CAPNPC} ${CAPNP} ${KJ}) + if(KJ AND CAPNP AND CAPNPC) + set (CAPNP_LIBRARIES ${CAPNPC} ${CAPNP} ${KJ}) + else() + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system capnproto") + endif() +endif() + +if (CAPNP_LIBRARIES) + set (USE_CAPNP 1) elseif(NOT MISSING_INTERNAL_CAPNP_LIBRARY) add_subdirectory(contrib/capnproto-cmake) set (CAPNP_LIBRARIES capnpc) -endif () - -if (CAPNP_LIBRARIES) set (USE_CAPNP 1) -endif () - + set (USE_INTERNAL_CAPNP_LIBRARY 1) endif () message (STATUS "Using capnp=${USE_CAPNP}: ${CAPNP_LIBRARIES}") diff --git a/cmake/find/cassandra.cmake b/cmake/find/cassandra.cmake index f41e0f645f4..e28f1534c09 100644 --- a/cmake/find/cassandra.cmake +++ b/cmake/find/cassandra.cmake @@ -1,25 +1,29 @@ option(ENABLE_CASSANDRA "Enable Cassandra" ${ENABLE_LIBRARIES}) -if (ENABLE_CASSANDRA) - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libuv") - message (ERROR "submodule contrib/libuv is missing. to fix try run: \n git submodule update --init --recursive") - elseif (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cassandra") - message (ERROR "submodule contrib/cassandra is missing. to fix try run: \n git submodule update --init --recursive") - else() - set (LIBUV_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/libuv") - set (CASSANDRA_INCLUDE_DIR - "${ClickHouse_SOURCE_DIR}/contrib/cassandra/include/") - if (USE_STATIC_LIBRARIES) - set (LIBUV_LIBRARY uv_a) - set (CASSANDRA_LIBRARY cassandra_static) - else() - set (LIBUV_LIBRARY uv) - set (CASSANDRA_LIBRARY cassandra) - endif() - set (USE_CASSANDRA 1) - set (CASS_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/cassandra") +if (NOT ENABLE_CASSANDRA) + return() +endif() +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libuv") + message (ERROR "submodule contrib/libuv is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libuv needed for Cassandra") +elseif (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cassandra") + message (ERROR "submodule contrib/cassandra is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal Cassandra") +else() + set (LIBUV_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/libuv") + set (CASSANDRA_INCLUDE_DIR + "${ClickHouse_SOURCE_DIR}/contrib/cassandra/include/") + if (MAKE_STATIC_LIBRARIES) + set (LIBUV_LIBRARY uv_a) + set (CASSANDRA_LIBRARY cassandra_static) + else() + set (LIBUV_LIBRARY uv) + set (CASSANDRA_LIBRARY cassandra) endif() + + set (USE_CASSANDRA 1) + set (CASS_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/cassandra") endif() message (STATUS "Using cassandra=${USE_CASSANDRA}: ${CASSANDRA_INCLUDE_DIR} : ${CASSANDRA_LIBRARY}") diff --git a/cmake/find/ccache.cmake b/cmake/find/ccache.cmake index 95d6b208cfa..e20e6416e97 100644 --- a/cmake/find/ccache.cmake +++ b/cmake/find/ccache.cmake @@ -1,3 +1,9 @@ +option(ENABLE_CCACHE "Speedup re-compilations using ccache" ON) + +if (NOT ENABLE_CCACHE) + return() +endif() + find_program (CCACHE_FOUND ccache) if (CCACHE_FOUND AND NOT CMAKE_CXX_COMPILER_LAUNCHER MATCHES "ccache" AND NOT CMAKE_CXX_COMPILER MATCHES "ccache") @@ -9,6 +15,8 @@ if (CCACHE_FOUND AND NOT CMAKE_CXX_COMPILER_LAUNCHER MATCHES "ccache" AND NOT CM set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE ${CCACHE_FOUND}) set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK ${CCACHE_FOUND}) else () - message(STATUS "Not using ${CCACHE_FOUND} ${CCACHE_VERSION} bug: https://bugzilla.samba.org/show_bug.cgi?id=8118") + message(${RECONFIGURE_MESSAGE_LEVEL} "Not using ${CCACHE_FOUND} ${CCACHE_VERSION} bug: https://bugzilla.samba.org/show_bug.cgi?id=8118") endif () +elseif (NOT CCACHE_FOUND) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use ccache") endif () diff --git a/cmake/find/cxx.cmake b/cmake/find/cxx.cmake index f6c999351e8..02f7113e6fb 100644 --- a/cmake/find/cxx.cmake +++ b/cmake/find/cxx.cmake @@ -1,41 +1,68 @@ -set(USE_INTERNAL_LIBCXX_LIBRARY_DEFAULT ${NOT_UNBUNDLED}) +option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++" ${NOT_UNBUNDLED}) -if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libcxx/CMakeLists.txt") - message(WARNING "submodule contrib/libcxx is missing. to fix try run: \n git submodule update --init --recursive") - set(USE_INTERNAL_LIBCXX_LIBRARY_DEFAULT 0) +if (NOT USE_LIBCXX) + if (USE_INTERNAL_LIBCXX_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal libcxx with USE_LIBCXX=OFF") + endif() + target_link_libraries(global-libs INTERFACE -l:libstdc++.a -l:libstdc++fs.a) # Always link these libraries as static + target_link_libraries(global-libs INTERFACE ${EXCEPTION_HANDLING_LIBRARY}) + return() endif() -option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++" ${NOT_UNBUNDLED}) +set(USE_INTERNAL_LIBCXX_LIBRARY_DEFAULT ${NOT_UNBUNDLED}) option (USE_INTERNAL_LIBCXX_LIBRARY "Set to FALSE to use system libcxx and libcxxabi libraries instead of bundled" ${USE_INTERNAL_LIBCXX_LIBRARY_DEFAULT}) -if (USE_LIBCXX) - set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=0") # More checks in debug build. +if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libcxx/CMakeLists.txt") + if (USE_INTERNAL_LIBCXX_LIBRARY) + message(WARNING "submodule contrib/libcxx is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libcxx") + set(USE_INTERNAL_LIBCXX_LIBRARY 0) + endif() + set(USE_INTERNAL_LIBCXX_LIBRARY_DEFAULT 0) + set(MISSING_INTERNAL_LIBCXX_LIBRARY 1) +endif() - if (NOT USE_INTERNAL_LIBCXX_LIBRARY) - find_library (LIBCXX_LIBRARY c++) - find_library (LIBCXXFS_LIBRARY c++fs) - find_library (LIBCXXABI_LIBRARY c++abi) +set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=0") # More checks in debug build. - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") +if (NOT USE_INTERNAL_LIBCXX_LIBRARY) + find_library (LIBCXX_LIBRARY c++) + find_library (LIBCXXFS_LIBRARY c++fs) + find_library (LIBCXXABI_LIBRARY c++abi) - target_link_libraries(global-libs INTERFACE ${EXCEPTION_HANDLING_LIBRARY}) + if(LIBCXX_LIBRARY AND LIBCXXABI_LIBRARY) # c++fs is now a part of the libc++ + set (HAVE_LIBCXX 1) else () - set (LIBCXX_LIBRARY cxx) - set (LIBCXXABI_LIBRARY cxxabi) - add_subdirectory(contrib/libcxxabi-cmake) - add_subdirectory(contrib/libcxx-cmake) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system libcxx") + endif() - # Exception handling library is embedded into libcxxabi. - endif () + if(NOT LIBCXXFS_LIBRARY) + set(LIBCXXFS_LIBRARY ${LIBCXX_LIBRARY}) + endif() - target_link_libraries(global-libs INTERFACE ${LIBCXX_LIBRARY} ${LIBCXXABI_LIBRARY} ${LIBCXXFS_LIBRARY}) + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") + + target_link_libraries(global-libs INTERFACE ${EXCEPTION_HANDLING_LIBRARY}) +endif () + +if (NOT HAVE_LIBCXX AND NOT MISSING_INTERNAL_LIBCXX_LIBRARY) + set (LIBCXX_LIBRARY cxx) + set (LIBCXXABI_LIBRARY cxxabi) + add_subdirectory(contrib/libcxxabi-cmake) + add_subdirectory(contrib/libcxx-cmake) + + # Exception handling library is embedded into libcxxabi. set (HAVE_LIBCXX 1) + set(USE_INTERNAL_LIBCXX_LIBRARY 1) +endif () + +if (HAVE_LIBCXX) + target_link_libraries(global-libs INTERFACE ${LIBCXX_LIBRARY} ${LIBCXXABI_LIBRARY} ${LIBCXXFS_LIBRARY}) message (STATUS "Using libcxx: ${LIBCXX_LIBRARY}") message (STATUS "Using libcxxfs: ${LIBCXXFS_LIBRARY}") message (STATUS "Using libcxxabi: ${LIBCXXABI_LIBRARY}") -else () +else() target_link_libraries(global-libs INTERFACE -l:libstdc++.a -l:libstdc++fs.a) # Always link these libraries as static target_link_libraries(global-libs INTERFACE ${EXCEPTION_HANDLING_LIBRARY}) -endif () +endif() diff --git a/cmake/find/fastops.cmake b/cmake/find/fastops.cmake index 1296ba586a0..5ab320bdb7a 100644 --- a/cmake/find/fastops.cmake +++ b/cmake/find/fastops.cmake @@ -1,19 +1,24 @@ if(NOT ARCH_ARM AND NOT OS_FREEBSD AND NOT OS_DARWIN) option(ENABLE_FASTOPS "Enable fast vectorized mathematical functions library by Mikhail Parakhin" ${ENABLE_LIBRARIES}) +elseif(ENABLE_FASTOPS) + message (${RECONFIGURE_MESSAGE_LEVEL} "Fastops library is not supported on ARM, FreeBSD and Darwin") endif() -if(ENABLE_FASTOPS) - if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/fastops/fastops/fastops.h") - message(WARNING "submodule contrib/fastops is missing. to fix try run: \n git submodule update --init --recursive") - set(MISSING_INTERNAL_FASTOPS_LIBRARY 1) - endif() - if(NOT MISSING_INTERNAL_FASTOPS_LIBRARY) - set(USE_FASTOPS 1) - set(FASTOPS_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/fastops/) - set(FASTOPS_LIBRARY fastops) - endif() -else() +if(NOT ENABLE_FASTOPS) set(USE_FASTOPS 0) + return() +endif() + +if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/fastops/fastops/fastops.h") + message(WARNING "submodule contrib/fastops is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal fastops library") + set(MISSING_INTERNAL_FASTOPS_LIBRARY 1) +endif() + +if(NOT MISSING_INTERNAL_FASTOPS_LIBRARY) + set(USE_FASTOPS 1) + set(FASTOPS_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/fastops/) + set(FASTOPS_LIBRARY fastops) endif() message(STATUS "Using fastops=${USE_FASTOPS}: ${FASTOPS_INCLUDE_DIR} : ${FASTOPS_LIBRARY}") diff --git a/cmake/find/gperf.cmake b/cmake/find/gperf.cmake index 451f5c21571..9b806598c57 100644 --- a/cmake/find/gperf.cmake +++ b/cmake/find/gperf.cmake @@ -1,8 +1,11 @@ -# Check if gperf was installed -find_program(GPERF gperf) -if(GPERF) - option(ENABLE_GPERF "Use gperf function hash generator tool" ${ENABLE_LIBRARIES}) +if(NOT DEFINED ENABLE_GPERF OR ENABLE_GPERF) + # Check if gperf was installed + find_program(GPERF gperf) + if(GPERF) + option(ENABLE_GPERF "Use gperf function hash generator tool" ${ENABLE_LIBRARIES}) + endif() endif() + if (ENABLE_GPERF) if(NOT GPERF) message(FATAL_ERROR "Could not find the program gperf") diff --git a/cmake/find/grpc.cmake b/cmake/find/grpc.cmake index 0019dbd5eed..216f8ae1878 100644 --- a/cmake/find/grpc.cmake +++ b/cmake/find/grpc.cmake @@ -1,26 +1,38 @@ option (ENABLE_GRPC "Use gRPC" ${ENABLE_LIBRARIES}) -if (ENABLE_GRPC) - option (USE_INTERNAL_GRPC_LIBRARY "Set to FALSE to use system gRPC library instead of bundled" ${NOT_UNBUNDLED}) - +if (NOT ENABLE_GRPC) if (USE_INTERNAL_GRPC_LIBRARY) - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/grpc/include/grpc++/grpc++.h") - message(WARNING "submodule contrib/grpc is missing. To fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_GRPC_LIBRARY OFF) - elif (NOT USE_PROTOBUF) - message(WARNING "gRPC requires protobuf which is disabled") - set (USE_INTERNAL_GRPC_LIBRARY OFF) - else() - set (GRPC_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/grpc/include") - set (GRPC_LIBRARY "libgrpc++") - set (USE_GRPC ON) - endif() + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal gRPC library with ENABLE_GRPC=OFF") + endif() + return() +endif() + +option (USE_INTERNAL_GRPC_LIBRARY "Set to FALSE to use system gRPC library instead of bundled" ${NOT_UNBUNDLED}) + +if (NOT USE_INTERNAL_GRPC_LIBRARY) + find_package(grpc) + if (GRPC_INCLUDE_DIR AND GRPC_LIBRARY) + set (USE_GRPC ON) else() - find_package(grpc) - if (GRPC_INCLUDE_DIR AND GRPC_LIBRARY) - set (USE_GRPC ON) - endif() + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system gRPC") endif() endif() -message(STATUS "Using gRPC=${USE_GRPC}: ${GRPC_INCLUDE_DIR} : ${GRPC_LIBRARY}") +if (NOT USE_GRPC) + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/grpc/include/grpc++/grpc++.h") + message (WARNING "submodule contrib/grpc is missing. To fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal gRPC") + set (USE_INTERNAL_GRPC_LIBRARY OFF) + elseif (NOT USE_PROTOBUF) + message (WARNING "gRPC requires protobuf which is disabled") + message (${RECONFIGURE_MESSAGE_LEVEL} "Will not use internal gRPC without protobuf") + set (USE_INTERNAL_GRPC_LIBRARY OFF) + else() + set (GRPC_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/grpc/include") + set (GRPC_LIBRARY "libgrpc++") + set (USE_GRPC ON) + set (USE_INTERNAL_GRPC_LIBRARY ON) + endif() +endif() + +message (STATUS "Using gRPC=${USE_GRPC}: ${GRPC_INCLUDE_DIR} : ${GRPC_LIBRARY}") diff --git a/cmake/find/gtest.cmake b/cmake/find/gtest.cmake index b41c4cc0af8..36e45a1381e 100644 --- a/cmake/find/gtest.cmake +++ b/cmake/find/gtest.cmake @@ -1,22 +1,30 @@ option (ENABLE_GTEST_LIBRARY "Enable gtest library" ${ENABLE_LIBRARIES}) -if (ENABLE_GTEST_LIBRARY) +if (NOT ENABLE_GTEST_LIBRARY) + if(USE_INTERNAL_GTEST_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal Google Test when ENABLE_GTEST_LIBRARY=OFF") + endif() + return() +endif() option (USE_INTERNAL_GTEST_LIBRARY "Set to FALSE to use system Google Test instead of bundled" ${NOT_UNBUNDLED}) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest/CMakeLists.txt") if (USE_INTERNAL_GTEST_LIBRARY) message (WARNING "submodule contrib/googletest is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal gtest") set (USE_INTERNAL_GTEST_LIBRARY 0) endif () set (MISSING_INTERNAL_GTEST_LIBRARY 1) endif () - if(NOT USE_INTERNAL_GTEST_LIBRARY) # TODO: autodetect of GTEST_SRC_DIR by EXISTS /usr/src/googletest/CMakeLists.txt if(NOT GTEST_SRC_DIR) find_package(GTest) + if (NOT GTEST_INCLUDE_DIRS) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system Google Test") + endif() endif() endif() @@ -26,12 +34,13 @@ if (NOT GTEST_SRC_DIR AND NOT GTEST_INCLUDE_DIRS AND NOT MISSING_INTERNAL_GTEST_ set (GTEST_LIBRARIES gtest) set (GTEST_BOTH_LIBRARIES ${GTEST_MAIN_LIBRARIES} ${GTEST_LIBRARIES}) set (GTEST_INCLUDE_DIRS ${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest) +elseif(USE_INTERNAL_GTEST_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Wouldn't use internal Google Test library") + set (USE_INTERNAL_GTEST_LIBRARY 0) endif () if((GTEST_INCLUDE_DIRS AND GTEST_BOTH_LIBRARIES) OR GTEST_SRC_DIR) set(USE_GTEST 1) endif() -endif() - message (STATUS "Using gtest=${USE_GTEST}: ${GTEST_INCLUDE_DIRS} : ${GTEST_BOTH_LIBRARIES} : ${GTEST_SRC_DIR}") diff --git a/cmake/find/h3.cmake b/cmake/find/h3.cmake index e01f0269507..7ca070b45fe 100644 --- a/cmake/find/h3.cmake +++ b/cmake/find/h3.cmake @@ -1,28 +1,38 @@ option (ENABLE_H3 "Enable H3" ${ENABLE_LIBRARIES}) -if (ENABLE_H3) +if(NOT ENABLE_H3) + if(USE_INTERNAL_H3_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal h3 library with ENABLE_H3=OFF") + endif () + return() +endif() -option (USE_INTERNAL_H3_LIBRARY "Set to FALSE to use system h3 library instead of bundled" ${NOT_UNBUNDLED}) +option(USE_INTERNAL_H3_LIBRARY "Set to FALSE to use system h3 library instead of bundled" ${NOT_UNBUNDLED}) if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/h3/src/h3lib/include/h3Index.h") if(USE_INTERNAL_H3_LIBRARY) - message(WARNING "submodule contrib/h3 is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/h3 is missing. to fix try run: \n git submodule update --init --recursive") + message(${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal h3 library") + set(USE_INTERNAL_H3_LIBRARY 0) endif() set(MISSING_INTERNAL_H3_LIBRARY 1) - set(USE_INTERNAL_H3_LIBRARY 0) endif() -if (USE_INTERNAL_H3_LIBRARY) - set (H3_LIBRARY h3) - set (H3_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/h3/src/h3lib/include) -elseif (NOT MISSING_INTERNAL_H3_LIBRARY) - find_library (H3_LIBRARY h3) - find_path (H3_INCLUDE_DIR NAMES h3/h3api.h PATHS ${H3_INCLUDE_PATHS}) -endif () +if(NOT USE_INTERNAL_H3_LIBRARY) + find_library(H3_LIBRARY h3) + find_path(H3_INCLUDE_DIR NAMES h3/h3api.h PATHS ${H3_INCLUDE_PATHS}) + + if(NOT H3_LIBRARY OR NOT H3_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system h3 library") + endif() +endif() if (H3_LIBRARY AND H3_INCLUDE_DIR) set (USE_H3 1) -endif () - -endif () +elseif(NOT MISSING_INTERNAL_H3_LIBRARY) + set (H3_LIBRARY h3) + set (H3_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/h3/src/h3lib/include") + set (USE_H3 1) + set (USE_INTERNAL_H3_LIBRARY 1) +endif() message (STATUS "Using h3=${USE_H3}: ${H3_INCLUDE_DIR} : ${H3_LIBRARY}") diff --git a/cmake/find/hdfs3.cmake b/cmake/find/hdfs3.cmake index 6a0d01d2245..e8e9fb359c9 100644 --- a/cmake/find/hdfs3.cmake +++ b/cmake/find/hdfs3.cmake @@ -1,21 +1,33 @@ if(NOT ARCH_ARM AND NOT OS_FREEBSD AND NOT APPLE AND USE_PROTOBUF) option(ENABLE_HDFS "Enable HDFS" ${ENABLE_LIBRARIES}) +elseif(ENABLE_HDFS OR USE_INTERNAL_HDFS3_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use HDFS3 with current configuration") +endif() + +if(NOT ENABLE_HDFS) + if(USE_INTERNAL_HDFS3_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal HDFS3 library with ENABLE_HDFS3=OFF") + endif() + return() endif() -if(ENABLE_HDFS) option(USE_INTERNAL_HDFS3_LIBRARY "Set to FALSE to use system HDFS3 instead of bundled" ${NOT_UNBUNDLED}) if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libhdfs3/include/hdfs/hdfs.h") if(USE_INTERNAL_HDFS3_LIBRARY) - message(WARNING "submodule contrib/libhdfs3 is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/libhdfs3 is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal HDFS3 library") + set(USE_INTERNAL_HDFS3_LIBRARY 0) endif() set(MISSING_INTERNAL_HDFS3_LIBRARY 1) - set(USE_INTERNAL_HDFS3_LIBRARY 0) endif() if(NOT USE_INTERNAL_HDFS3_LIBRARY) find_library(HDFS3_LIBRARY hdfs3) find_path(HDFS3_INCLUDE_DIR NAMES hdfs/hdfs.h PATHS ${HDFS3_INCLUDE_PATHS}) + if(NOT HDFS3_LIBRARY OR NOT HDFS3_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot find system HDFS3 library") + endif() endif() if(HDFS3_LIBRARY AND HDFS3_INCLUDE_DIR) @@ -26,9 +38,7 @@ elseif(NOT MISSING_INTERNAL_HDFS3_LIBRARY AND LIBGSASL_LIBRARY AND LIBXML2_LIBRA set(USE_INTERNAL_HDFS3_LIBRARY 1) set(USE_HDFS 1) else() - set(USE_INTERNAL_HDFS3_LIBRARY 0) -endif() - + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannout enable HDFS3") endif() message(STATUS "Using hdfs3=${USE_HDFS}: ${HDFS3_INCLUDE_DIR} : ${HDFS3_LIBRARY}") diff --git a/cmake/find/icu.cmake b/cmake/find/icu.cmake index 7beb25626b9..40fb391656d 100644 --- a/cmake/find/icu.cmake +++ b/cmake/find/icu.cmake @@ -4,13 +4,20 @@ else () option(ENABLE_ICU "Enable ICU" 0) endif () -if (ENABLE_ICU) +if (NOT ENABLE_ICU) + if(USE_INTERNAL_ICU_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal icu library with ENABLE_ICU=OFF") + endif() + message(STATUS "Build without ICU (support for collations and charset conversion functions will be disabled)") + return() +endif() option (USE_INTERNAL_ICU_LIBRARY "Set to FALSE to use system ICU library instead of bundled" ${NOT_UNBUNDLED}) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/icu/icu4c/LICENSE") if (USE_INTERNAL_ICU_LIBRARY) message (WARNING "submodule contrib/icu is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal ICU") set (USE_INTERNAL_ICU_LIBRARY 0) endif () set (MISSING_INTERNAL_ICU_LIBRARY 1) @@ -24,6 +31,8 @@ if(NOT USE_INTERNAL_ICU_LIBRARY) #set (ICU_LIBRARIES ${ICU_I18N_LIBRARY} ${ICU_UC_LIBRARY} ${ICU_DATA_LIBRARY} CACHE STRING "") if(ICU_FOUND) set(USE_ICU 1) + else() + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system ICU") endif() endif() @@ -35,8 +44,6 @@ elseif (NOT MISSING_INTERNAL_ICU_LIBRARY) set (USE_ICU 1) endif () -endif() - if(USE_ICU) message(STATUS "Using icu=${USE_ICU}: ${ICU_INCLUDE_DIR} : ${ICU_LIBRARIES}") else() diff --git a/cmake/find/ldap.cmake b/cmake/find/ldap.cmake index 99c9007d6b5..3f7871617d4 100644 --- a/cmake/find/ldap.cmake +++ b/cmake/find/ldap.cmake @@ -1,84 +1,97 @@ option (ENABLE_LDAP "Enable LDAP" ${ENABLE_LIBRARIES}) -if (ENABLE_LDAP) - option (USE_INTERNAL_LDAP_LIBRARY "Set to FALSE to use system *LDAP library instead of bundled" ${NOT_UNBUNDLED}) +if (NOT ENABLE_LDAP) + if(USE_INTERNAL_LDAP_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal LDAP library with ENABLE_LDAP=OFF") + endif () + return() +endif() - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/openldap/README") - if (USE_INTERNAL_LDAP_LIBRARY) - message (WARNING "Submodule contrib/openldap is missing. To fix try running:\n git submodule update --init --recursive") - endif () +option (USE_INTERNAL_LDAP_LIBRARY "Set to FALSE to use system *LDAP library instead of bundled" ${NOT_UNBUNDLED}) - set (USE_INTERNAL_LDAP_LIBRARY 0) - set (MISSING_INTERNAL_LDAP_LIBRARY 1) +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/openldap/README") + if (USE_INTERNAL_LDAP_LIBRARY) + message (WARNING "Submodule contrib/openldap is missing. To fix try running:\n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal LDAP library") endif () - set (OPENLDAP_USE_STATIC_LIBS ${USE_STATIC_LIBRARIES}) - set (OPENLDAP_USE_REENTRANT_LIBS 1) + set (USE_INTERNAL_LDAP_LIBRARY 0) + set (MISSING_INTERNAL_LDAP_LIBRARY 1) +endif () - if (NOT USE_INTERNAL_LDAP_LIBRARY) - if (OPENLDAP_USE_STATIC_LIBS) - message (WARNING "Unable to use external static OpenLDAP libraries, falling back to the bundled version.") - set (USE_INTERNAL_LDAP_LIBRARY 1) - else () - if (APPLE AND NOT OPENLDAP_ROOT_DIR) - set (OPENLDAP_ROOT_DIR "/usr/local/opt/openldap") - endif () +set (OPENLDAP_USE_STATIC_LIBS ${USE_STATIC_LIBRARIES}) +set (OPENLDAP_USE_REENTRANT_LIBS 1) - find_package (OpenLDAP) - endif () - endif () - - if (NOT OPENLDAP_FOUND AND NOT MISSING_INTERNAL_LDAP_LIBRARY) - string (TOLOWER "${CMAKE_SYSTEM_NAME}" _system_name) - string (TOLOWER "${CMAKE_SYSTEM_PROCESSOR}" _system_processor) - - if ( - "${_system_processor}" STREQUAL "amd64" OR - "${_system_processor}" STREQUAL "x64" - ) - set (_system_processor "x86_64") - elseif ( - "${_system_processor}" STREQUAL "arm64" - ) - set (_system_processor "aarch64") +if (NOT USE_INTERNAL_LDAP_LIBRARY) + if (OPENLDAP_USE_STATIC_LIBS) + message (WARNING "Unable to use external static OpenLDAP libraries, falling back to the bundled version.") + message (${RECONFIGURE_MESSAGE_LEVEL} "Unable to use external OpenLDAP") + set (USE_INTERNAL_LDAP_LIBRARY 1) + else () + if (APPLE AND NOT OPENLDAP_ROOT_DIR) + set (OPENLDAP_ROOT_DIR "/usr/local/opt/openldap") endif () - if ( - ( "${_system_name}" STREQUAL "linux" AND "${_system_processor}" STREQUAL "x86_64" ) OR - ( "${_system_name}" STREQUAL "linux" AND "${_system_processor}" STREQUAL "aarch64" ) OR - ( "${_system_name}" STREQUAL "freebsd" AND "${_system_processor}" STREQUAL "x86_64" ) OR - ( "${_system_name}" STREQUAL "darwin" AND "${_system_processor}" STREQUAL "x86_64" ) - ) - set (_ldap_supported_platform TRUE) - endif () + find_package (OpenLDAP) - if (NOT _ldap_supported_platform) - message (WARNING "LDAP support using the bundled library is not implemented for ${CMAKE_SYSTEM_NAME} ${CMAKE_SYSTEM_PROCESSOR} platform.") - elseif (NOT USE_SSL) - message (WARNING "LDAP support using the bundled library is not possible if SSL is not used.") - else () - set (USE_INTERNAL_LDAP_LIBRARY 1) - set (OPENLDAP_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/openldap") - set (OPENLDAP_INCLUDE_DIRS - "${ClickHouse_SOURCE_DIR}/contrib/openldap-cmake/${_system_name}_${_system_processor}/include" - "${ClickHouse_SOURCE_DIR}/contrib/openldap/include" - ) - # Below, 'ldap'/'ldap_r' and 'lber' will be resolved to - # the targets defined in contrib/openldap-cmake/CMakeLists.txt - if (OPENLDAP_USE_REENTRANT_LIBS) - set (OPENLDAP_LDAP_LIBRARY "ldap_r") - else () - set (OPENLDAP_LDAP_LIBRARY "ldap") - endif() - set (OPENLDAP_LBER_LIBRARY "lber") - set (OPENLDAP_LIBRARIES ${OPENLDAP_LDAP_LIBRARY} ${OPENLDAP_LBER_LIBRARY}) - set (OPENLDAP_FOUND 1) - endif () - endif () - - if (OPENLDAP_FOUND) - set (USE_LDAP 1) + if (NOT OPENLDAP_FOUND) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system OpenLDAP") + endif() endif () endif () +if (NOT OPENLDAP_FOUND AND NOT MISSING_INTERNAL_LDAP_LIBRARY) + string (TOLOWER "${CMAKE_SYSTEM_NAME}" _system_name) + string (TOLOWER "${CMAKE_SYSTEM_PROCESSOR}" _system_processor) + + if ( + "${_system_processor}" STREQUAL "amd64" OR + "${_system_processor}" STREQUAL "x64" + ) + set (_system_processor "x86_64") + elseif ( + "${_system_processor}" STREQUAL "arm64" + ) + set (_system_processor "aarch64") + endif () + + if ( + ( "${_system_name}" STREQUAL "linux" AND "${_system_processor}" STREQUAL "x86_64" ) OR + ( "${_system_name}" STREQUAL "linux" AND "${_system_processor}" STREQUAL "aarch64" ) OR + ( "${_system_name}" STREQUAL "freebsd" AND "${_system_processor}" STREQUAL "x86_64" ) OR + ( "${_system_name}" STREQUAL "darwin" AND "${_system_processor}" STREQUAL "x86_64" ) + ) + set (_ldap_supported_platform TRUE) + endif () + + if (NOT _ldap_supported_platform) + message (WARNING "LDAP support using the bundled library is not implemented for ${CMAKE_SYSTEM_NAME} ${CMAKE_SYSTEM_PROCESSOR} platform.") + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot enable LDAP support") + elseif (NOT USE_SSL) + message (WARNING "LDAP support using the bundled library is not possible if SSL is not used.") + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot enable LDAP support") + else () + set (USE_INTERNAL_LDAP_LIBRARY 1) + set (OPENLDAP_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/openldap") + set (OPENLDAP_INCLUDE_DIRS + "${ClickHouse_SOURCE_DIR}/contrib/openldap-cmake/${_system_name}_${_system_processor}/include" + "${ClickHouse_SOURCE_DIR}/contrib/openldap/include" + ) + # Below, 'ldap'/'ldap_r' and 'lber' will be resolved to + # the targets defined in contrib/openldap-cmake/CMakeLists.txt + if (OPENLDAP_USE_REENTRANT_LIBS) + set (OPENLDAP_LDAP_LIBRARY "ldap_r") + else () + set (OPENLDAP_LDAP_LIBRARY "ldap") + endif() + set (OPENLDAP_LBER_LIBRARY "lber") + set (OPENLDAP_LIBRARIES ${OPENLDAP_LDAP_LIBRARY} ${OPENLDAP_LBER_LIBRARY}) + set (OPENLDAP_FOUND 1) + endif () +endif () + +if (OPENLDAP_FOUND) + set (USE_LDAP 1) +endif () + message (STATUS "Using ldap=${USE_LDAP}: ${OPENLDAP_INCLUDE_DIRS} : ${OPENLDAP_LIBRARIES}") diff --git a/cmake/find/libgsasl.cmake b/cmake/find/libgsasl.cmake index e9c45a09010..3b580a4ddb2 100644 --- a/cmake/find/libgsasl.cmake +++ b/cmake/find/libgsasl.cmake @@ -1,12 +1,18 @@ option(ENABLE_GSASL_LIBRARY "Enable gsasl library" ${ENABLE_LIBRARIES}) -if (ENABLE_GSASL_LIBRARY) +if (NOT ENABLE_GSASL_LIBRARY) + if(USE_INTERNAL_LIBGSASL_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal libgsasl library with ENABLE_GSASL_LIBRARY=OFF") + endif() + return() +endif() option (USE_INTERNAL_LIBGSASL_LIBRARY "Set to FALSE to use system libgsasl library instead of bundled" ${NOT_UNBUNDLED}) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src/gsasl.h") if (USE_INTERNAL_LIBGSASL_LIBRARY) message (WARNING "submodule contrib/libgsasl is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libgsasl") set (USE_INTERNAL_LIBGSASL_LIBRARY 0) endif () set (MISSING_INTERNAL_LIBGSASL_LIBRARY 1) @@ -14,12 +20,15 @@ endif () if (NOT USE_INTERNAL_LIBGSASL_LIBRARY) find_library (LIBGSASL_LIBRARY gsasl) - find_path (LIBGSASL_INCLUDE_DIR NAMES gsasl.h PATHS ${LIBGSASL_INCLUDE_PATHS}) + find_path (LIBGSASL_INCLUDE_DIR NAMES gsasl.h PATHS ${LIBGSASL_INCLUDE_PATHS}) + if (NOT LIBGSASL_LIBRARY OR NOT LIBGSASL_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system libgsasl") + endif () endif () if (LIBGSASL_LIBRARY AND LIBGSASL_INCLUDE_DIR) elseif (NOT MISSING_INTERNAL_LIBGSASL_LIBRARY) - set (LIBGSASL_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src ${ClickHouse_SOURCE_DIR}/contrib/libgsasl/linux_x86_64/include) + set (LIBGSASL_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src" "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/linux_x86_64/include") set (USE_INTERNAL_LIBGSASL_LIBRARY 1) set (LIBGSASL_LIBRARY libgsasl) endif () @@ -28,6 +37,4 @@ if(LIBGSASL_LIBRARY AND LIBGSASL_INCLUDE_DIR) set (USE_LIBGSASL 1) endif() -endif() - message (STATUS "Using libgsasl=${USE_LIBGSASL}: ${LIBGSASL_INCLUDE_DIR} : ${LIBGSASL_LIBRARY}") diff --git a/cmake/find/libxml2.cmake b/cmake/find/libxml2.cmake index 73732e139c5..04609e497a8 100644 --- a/cmake/find/libxml2.cmake +++ b/cmake/find/libxml2.cmake @@ -3,6 +3,7 @@ option (USE_INTERNAL_LIBXML2_LIBRARY "Set to FALSE to use system libxml2 library if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libxml2/libxml.h") if (USE_INTERNAL_LIBXML2_LIBRARY) message (WARNING "submodule contrib/libxml2 is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libxml") set (USE_INTERNAL_LIBXML2_LIBRARY 0) endif () set (MISSING_INTERNAL_LIBXML2_LIBRARY 1) @@ -12,6 +13,10 @@ if (NOT USE_INTERNAL_LIBXML2_LIBRARY) find_package (LibXml2) #find_library (LIBXML2_LIBRARY libxml2) #find_path (LIBXML2_INCLUDE_DIR NAMES libxml.h PATHS ${LIBXML2_INCLUDE_PATHS}) + + if (NOT LIBXML2_LIBRARY OR NOT LIBXML2_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system libxml2") + endif () endif () if (LIBXML2_LIBRARY AND LIBXML2_INCLUDE_DIR) diff --git a/cmake/find/llvm.cmake b/cmake/find/llvm.cmake index 7cb67d1a990..e46b8c9335e 100644 --- a/cmake/find/llvm.cmake +++ b/cmake/find/llvm.cmake @@ -2,78 +2,89 @@ if (NOT APPLE) option (ENABLE_EMBEDDED_COMPILER "Set to TRUE to enable support for 'compile_expressions' option for query execution" ${ENABLE_LIBRARIES}) option (USE_INTERNAL_LLVM_LIBRARY "Use bundled or system LLVM library." ${NOT_UNBUNDLED}) +elseif(ENABLE_EMBEDDED_COMPILER OR USE_INTERNAL_LLVM_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Embedded compiler is not currently supported on Apple") endif () -if (ENABLE_EMBEDDED_COMPILER) - if (USE_INTERNAL_LLVM_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/llvm/llvm/CMakeLists.txt") - message (WARNING "submodule contrib/llvm is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_LLVM_LIBRARY 0) - endif () - - if (NOT USE_INTERNAL_LLVM_LIBRARY) - set (LLVM_PATHS "/usr/local/lib/llvm") - - foreach(llvm_v 9 8) - if (NOT LLVM_FOUND) - find_package (LLVM ${llvm_v} CONFIG PATHS ${LLVM_PATHS}) - endif () - endforeach () - - if (LLVM_FOUND) - # Remove dynamically-linked zlib and libedit from LLVM's dependencies: - set_target_properties(LLVMSupport PROPERTIES INTERFACE_LINK_LIBRARIES "-lpthread;LLVMDemangle;${ZLIB_LIBRARIES}") - set_target_properties(LLVMLineEditor PROPERTIES INTERFACE_LINK_LIBRARIES "LLVMSupport") - - option(LLVM_HAS_RTTI "Enable if LLVM was build with RTTI enabled" ON) - set (USE_EMBEDDED_COMPILER 1) - else() - set (USE_EMBEDDED_COMPILER 0) - endif() - - if (LLVM_FOUND AND OS_LINUX AND USE_LIBCXX) - message(WARNING "Option USE_INTERNAL_LLVM_LIBRARY is not set but the LLVM library from OS packages in Linux is incompatible with libc++ ABI. LLVM Will be disabled.") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - endif () - else() - if (CMAKE_CURRENT_SOURCE_DIR STREQUAL CMAKE_CURRENT_BINARY_DIR) - message(WARNING "Option ENABLE_EMBEDDED_COMPILER is set but LLVM library cannot build if build directory is the same as source directory.") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - elseif (SPLIT_SHARED_LIBRARIES) - # llvm-tablegen cannot find shared libraries that we build. Probably can be easily fixed. - message(WARNING "Option ENABLE_EMBEDDED_COMPILER is not compatible with SPLIT_SHARED_LIBRARIES. Build of LLVM will be disabled.") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - elseif (NOT ARCH_AMD64) - # It's not supported yet, but you can help. - message(WARNING "Option ENABLE_EMBEDDED_COMPILER is only available for x86_64. Build of LLVM will be disabled.") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - elseif (SANITIZE STREQUAL "undefined") - # llvm-tblgen, that is used during LLVM build, doesn't work with UBSan. - message(WARNING "Option ENABLE_EMBEDDED_COMPILER does not work with UBSan, because 'llvm-tblgen' tool from LLVM has undefined behaviour. Build of LLVM will be disabled.") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - else () - set (LLVM_FOUND 1) - set (USE_EMBEDDED_COMPILER 1) - set (LLVM_VERSION "9.0.0bundled") - set (LLVM_INCLUDE_DIRS - ${ClickHouse_SOURCE_DIR}/contrib/llvm/llvm/include - ${ClickHouse_BINARY_DIR}/contrib/llvm/llvm/include - ) - set (LLVM_LIBRARY_DIRS ${ClickHouse_BINARY_DIR}/contrib/llvm/llvm) - endif() +if (NOT ENABLE_EMBEDDED_COMPILER) + if(USE_INTERNAL_LLVM_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal LLVM library with ENABLE_EMBEDDED_COMPILER=OFF") endif() + return() +endif() + +if (USE_INTERNAL_LLVM_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/llvm/llvm/CMakeLists.txt") + message (WARNING "submodule contrib/llvm is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't fidd internal LLVM library") + set (USE_INTERNAL_LLVM_LIBRARY 0) +endif () + +if (NOT USE_INTERNAL_LLVM_LIBRARY) + set (LLVM_PATHS "/usr/local/lib/llvm") + + foreach(llvm_v 9 8) + if (NOT LLVM_FOUND) + find_package (LLVM ${llvm_v} CONFIG PATHS ${LLVM_PATHS}) + endif () + endforeach () if (LLVM_FOUND) - message(STATUS "LLVM include Directory: ${LLVM_INCLUDE_DIRS}") - message(STATUS "LLVM library Directory: ${LLVM_LIBRARY_DIRS}") - message(STATUS "LLVM C++ compiler flags: ${LLVM_CXXFLAGS}") + # Remove dynamically-linked zlib and libedit from LLVM's dependencies: + set_target_properties(LLVMSupport PROPERTIES INTERFACE_LINK_LIBRARIES "-lpthread;LLVMDemangle;${ZLIB_LIBRARIES}") + set_target_properties(LLVMLineEditor PROPERTIES INTERFACE_LINK_LIBRARIES "LLVMSupport") + + option(LLVM_HAS_RTTI "Enable if LLVM was build with RTTI enabled" ON) + set (USE_EMBEDDED_COMPILER 1) + else() + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system LLVM") + set (USE_EMBEDDED_COMPILER 0) + endif() + + if (LLVM_FOUND AND OS_LINUX AND USE_LIBCXX) + message(WARNING "Option USE_INTERNAL_LLVM_LIBRARY is not set but the LLVM library from OS packages in Linux is incompatible with libc++ ABI. LLVM Will be disabled.") + message (${RECONFIGURE_MESSAGE_LEVEL} "Unsupported LLVM configuration, cannot enable LLVM") + set (LLVM_FOUND 0) + set (USE_EMBEDDED_COMPILER 0) + endif () +else() + if (CMAKE_CURRENT_SOURCE_DIR STREQUAL CMAKE_CURRENT_BINARY_DIR) + message(WARNING "Option ENABLE_EMBEDDED_COMPILER is set but LLVM library cannot build if build directory is the same as source directory.") + set (LLVM_FOUND 0) + set (USE_EMBEDDED_COMPILER 0) + elseif (SPLIT_SHARED_LIBRARIES) + # llvm-tablegen cannot find shared libraries that we build. Probably can be easily fixed. + message(WARNING "Option ENABLE_EMBEDDED_COMPILER is not compatible with SPLIT_SHARED_LIBRARIES. Build of LLVM will be disabled.") + set (LLVM_FOUND 0) + set (USE_EMBEDDED_COMPILER 0) + elseif (NOT ARCH_AMD64) + # It's not supported yet, but you can help. + message(WARNING "Option ENABLE_EMBEDDED_COMPILER is only available for x86_64. Build of LLVM will be disabled.") + set (LLVM_FOUND 0) + set (USE_EMBEDDED_COMPILER 0) + elseif (SANITIZE STREQUAL "undefined") + # llvm-tblgen, that is used during LLVM build, doesn't work with UBSan. + message(WARNING "Option ENABLE_EMBEDDED_COMPILER does not work with UBSan, because 'llvm-tblgen' tool from LLVM has undefined behaviour. Build of LLVM will be disabled.") + set (LLVM_FOUND 0) + set (USE_EMBEDDED_COMPILER 0) + else () + set (LLVM_FOUND 1) + set (USE_EMBEDDED_COMPILER 1) + set (LLVM_VERSION "9.0.0bundled") + set (LLVM_INCLUDE_DIRS + ${ClickHouse_SOURCE_DIR}/contrib/llvm/llvm/include + ${ClickHouse_BINARY_DIR}/contrib/llvm/llvm/include + ) + set (LLVM_LIBRARY_DIRS ${ClickHouse_BINARY_DIR}/contrib/llvm/llvm) endif() endif() +if (LLVM_FOUND) + message(STATUS "LLVM include Directory: ${LLVM_INCLUDE_DIRS}") + message(STATUS "LLVM library Directory: ${LLVM_LIBRARY_DIRS}") + message(STATUS "LLVM C++ compiler flags: ${LLVM_CXXFLAGS}") +else() + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't enable LLVM") +endif() # This list was generated by listing all LLVM libraries, compiling the binary and removing all libraries while it still compiles. set (REQUIRED_LLVM_LIBRARIES diff --git a/cmake/find/ltdl.cmake b/cmake/find/ltdl.cmake index 18003618dbd..b48a3630222 100644 --- a/cmake/find/ltdl.cmake +++ b/cmake/find/ltdl.cmake @@ -1,5 +1,5 @@ if (ENABLE_ODBC AND NOT USE_INTERNAL_ODBC_LIBRARY) set (LTDL_PATHS "/usr/local/opt/libtool/lib") - find_library (LTDL_LIBRARY ltdl PATHS ${LTDL_PATHS}) + find_library (LTDL_LIBRARY ltdl PATHS ${LTDL_PATHS} REQUIRED) message (STATUS "Using ltdl: ${LTDL_LIBRARY}") endif () diff --git a/cmake/find/msgpack.cmake b/cmake/find/msgpack.cmake index 102ea619f6a..130aa007ad5 100644 --- a/cmake/find/msgpack.cmake +++ b/cmake/find/msgpack.cmake @@ -1,27 +1,37 @@ option (ENABLE_MSGPACK "Enable msgpack library" ${ENABLE_LIBRARIES}) -if (ENABLE_MSGPACK) +if(NOT ENABLE_MSGPACK) + if(USE_INTERNAL_MSGPACK_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal msgpack with ENABLE_MSGPACK=OFF") + endif() + return() +endif() option (USE_INTERNAL_MSGPACK_LIBRARY "Set to FALSE to use system msgpack library instead of bundled" ${NOT_UNBUNDLED}) -if (USE_INTERNAL_MSGPACK_LIBRARY) - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/msgpack-c/include/msgpack.hpp") +if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/msgpack-c/include/msgpack.hpp") + if(USE_INTERNAL_MSGPACK_LIBRARY) message(WARNING "Submodule contrib/msgpack-c is missing. To fix try run: \n git submodule update --init --recursive") - set(USE_INTERNAL_MSGPACK_LIBRARY 0) - set(MISSING_INTERNAL_MSGPACK_LIBRARY 1) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal msgpack") + set(USE_INTERNAL_MSGPACK_LIBRARY 0) + endif() + set(MISSING_INTERNAL_MSGPACK_LIBRARY 1) +endif() + +if(NOT USE_INTERNAL_MSGPACK_LIBRARY) + find_path(MSGPACK_INCLUDE_DIR NAMES msgpack.hpp PATHS ${MSGPACK_INCLUDE_PATHS}) + if(NOT MSGPACK_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system msgpack") endif() endif() -if (USE_INTERNAL_MSGPACK_LIBRARY) - set(MSGPACK_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/msgpack-c/include) -else() - find_path(MSGPACK_INCLUDE_DIR NAMES msgpack.hpp PATHS ${MSGPACK_INCLUDE_PATHS}) +if(NOT MSGPACK_INCLUDE_DIR AND NOT MISSING_INTERNAL_MSGPACK_LIBRARY) + set(MSGPACK_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/msgpack-c/include") + set(USE_INTERNAL_MSGPACK_LIBRARY 1) endif() if (MSGPACK_INCLUDE_DIR) set(USE_MSGPACK 1) endif() -endif() - message(STATUS "Using msgpack=${USE_MSGPACK}: ${MSGPACK_INCLUDE_DIR}") diff --git a/cmake/find/mysqlclient.cmake b/cmake/find/mysqlclient.cmake index 886c5b9bb59..b0bfd02fabc 100644 --- a/cmake/find/mysqlclient.cmake +++ b/cmake/find/mysqlclient.cmake @@ -4,56 +4,71 @@ else () option(ENABLE_MYSQL "Enable MySQL" FALSE) endif () -if(ENABLE_MYSQL) - option(USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" ${NOT_UNBUNDLED}) +if(NOT ENABLE_MYSQL) + if (USE_INTERNAL_MYSQL_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal mysql library with ENABLE_MYSQL=OFF") + endif () + message (STATUS "Build without mysqlclient (support for MYSQL dictionary source will be disabled)") + return() +endif() - if(USE_INTERNAL_MYSQL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README") +option(USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" ${NOT_UNBUNDLED}) + +if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README") + if(USE_INTERNAL_MYSQL_LIBRARY) message(WARNING "submodule contrib/mariadb-connector-c is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal mysql library") set(USE_INTERNAL_MYSQL_LIBRARY 0) endif() + set(MISSING_INTERNAL_MYSQL_LIBRARY 1) +endif() - if (USE_INTERNAL_MYSQL_LIBRARY) - set (MYSQLCLIENT_LIBRARIES mariadbclient) +if (NOT USE_INTERNAL_MYSQL_LIBRARY) + set (MYSQL_LIB_PATHS + "/usr/local/opt/mysql/lib" + "/usr/local/lib" + "/usr/local/lib64" + "/usr/local/lib/mariadb" # macos brew mariadb-connector-c + "/usr/mysql/lib" + "/usr/mysql/lib64" + "/usr/lib" + "/usr/lib64" + "/lib" + "/lib64") + + set (MYSQL_INCLUDE_PATHS + "/usr/local/opt/mysql/include" + "/usr/mysql/include" + "/usr/local/include" + "/usr/include") + + find_path (MYSQL_INCLUDE_DIR NAMES mysql/mysql.h mariadb/mysql.h PATHS ${MYSQL_INCLUDE_PATHS} PATH_SUFFIXES mysql) + + if (USE_STATIC_LIBRARIES) + find_library (STATIC_MYSQLCLIENT_LIB NAMES mariadbclient mysqlclient PATHS ${MYSQL_LIB_PATHS} PATH_SUFFIXES mysql) + else () + find_library (MYSQLCLIENT_LIBRARIES NAMES mariadb mariadbclient mysqlclient PATHS ${MYSQL_LIB_PATHS} PATH_SUFFIXES mysql) + endif () + + if (MYSQL_INCLUDE_DIR AND (STATIC_MYSQLCLIENT_LIB OR MYSQLCLIENT_LIBRARIES)) set (USE_MYSQL 1) set (MYSQLXX_LIBRARY mysqlxx) + if (APPLE) + # /usr/local/include/mysql/mysql_com.h:1011:10: fatal error: mysql/udf_registration_types.h: No such file or directory + set(MYSQL_INCLUDE_DIR ${MYSQL_INCLUDE_DIR} ${MYSQL_INCLUDE_DIR}/mysql) + endif () else () - set (MYSQL_LIB_PATHS - "/usr/local/opt/mysql/lib" - "/usr/local/lib" - "/usr/local/lib64" - "/usr/local/lib/mariadb" # macos brew mariadb-connector-c - "/usr/mysql/lib" - "/usr/mysql/lib64" - "/usr/lib" - "/usr/lib64" - "/lib" - "/lib64") - - set (MYSQL_INCLUDE_PATHS - "/usr/local/opt/mysql/include" - "/usr/mysql/include" - "/usr/local/include" - "/usr/include") - - find_path (MYSQL_INCLUDE_DIR NAMES mysql/mysql.h mariadb/mysql.h PATHS ${MYSQL_INCLUDE_PATHS} PATH_SUFFIXES mysql) - - if (USE_STATIC_LIBRARIES) - find_library (STATIC_MYSQLCLIENT_LIB NAMES mariadbclient mysqlclient PATHS ${MYSQL_LIB_PATHS} PATH_SUFFIXES mysql) - else () - find_library (MYSQLCLIENT_LIBRARIES NAMES mariadb mariadbclient mysqlclient PATHS ${MYSQL_LIB_PATHS} PATH_SUFFIXES mysql) - endif () - - if (MYSQL_INCLUDE_DIR AND (STATIC_MYSQLCLIENT_LIB OR MYSQLCLIENT_LIBRARIES)) - set (USE_MYSQL 1) - set (MYSQLXX_LIBRARY mysqlxx) - if (APPLE) - # /usr/local/include/mysql/mysql_com.h:1011:10: fatal error: mysql/udf_registration_types.h: No such file or directory - set(MYSQL_INCLUDE_DIR ${MYSQL_INCLUDE_DIR} ${MYSQL_INCLUDE_DIR}/mysql) - endif () - endif () + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system mysql library") endif () endif () +if (NOT USE_MYSQL AND NOT MISSING_INTERNAL_MYSQL_LIBRARY) + set (MYSQLCLIENT_LIBRARIES mariadbclient) + set (MYSQLXX_LIBRARY mysqlxx) + set (USE_MYSQL 1) + set (USE_INTERNAL_MYSQL_LIBRARY 1) +endif() + if (USE_MYSQL) message (STATUS "Using mysqlclient=${USE_MYSQL}: ${MYSQL_INCLUDE_DIR} : ${MYSQLCLIENT_LIBRARIES}; staticlib=${STATIC_MYSQLCLIENT_LIB}") else () diff --git a/cmake/find/opencl.cmake b/cmake/find/opencl.cmake index 0f307350cb8..2b0cc7c5dd4 100644 --- a/cmake/find/opencl.cmake +++ b/cmake/find/opencl.cmake @@ -3,12 +3,14 @@ if(0) option(ENABLE_OPENCL "Enable OpenCL support" ${ENABLE_LIBRARIES}) endif() -if(ENABLE_OPENCL) +if(NOT ENABLE_OPENCL) + return() +endif() # Intel OpenCl driver: sudo apt install intel-opencl-icd # @sa https://github.com/intel/compute-runtime/releases -# OpenCL applications should link wiht ICD loader +# OpenCL applications should link with ICD loader # sudo apt install opencl-headers ocl-icd-libopencl1 # sudo ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so # TODO: add https://github.com/OCL-dev/ocl-icd as submodule instead @@ -16,8 +18,8 @@ if(ENABLE_OPENCL) find_package(OpenCL) if(OpenCL_FOUND) set(USE_OPENCL 1) -endif() - +else() + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't enable OpenCL support") endif() message(STATUS "Using opencl=${USE_OPENCL}: ${OpenCL_INCLUDE_DIRS} : ${OpenCL_LIBRARIES}") diff --git a/cmake/find/orc.cmake b/cmake/find/orc.cmake index 26253687c80..c9474a095c4 100644 --- a/cmake/find/orc.cmake +++ b/cmake/find/orc.cmake @@ -1,12 +1,20 @@ option (ENABLE_ORC "Enable ORC" ${ENABLE_LIBRARIES}) -if(ENABLE_ORC) -include(cmake/find/snappy.cmake) +if(NOT ENABLE_ORC) + if(USE_INTERNAL_ORC_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal ORC library with ENABLE_ORD=OFF") + endif() + return() +endif() + option(USE_INTERNAL_ORC_LIBRARY "Set to FALSE to use system ORC instead of bundled" ${NOT_UNBUNDLED}) +include(cmake/find/snappy.cmake) + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/orc/c++/include/orc/OrcFile.hh") if(USE_INTERNAL_ORC_LIBRARY) message(WARNING "submodule contrib/orc is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal ORC") set(USE_INTERNAL_ORC_LIBRARY 0) endif() set(MISSING_INTERNAL_ORC_LIBRARY 1) @@ -14,6 +22,9 @@ endif () if (NOT USE_INTERNAL_ORC_LIBRARY) find_package(orc) + if (NOT ORC_LIBRARY OR NOT ORC_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system ORC") + endif () endif () #if (USE_INTERNAL_ORC_LIBRARY) @@ -30,10 +41,10 @@ elseif(NOT MISSING_INTERNAL_ORC_LIBRARY AND ARROW_LIBRARY AND SNAPPY_LIBRARY) # set(ORC_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/orc/c++/include") set(ORC_LIBRARY orc) set(USE_ORC 1) + set(USE_INTERNAL_ORC_LIBRARY 1) else() + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't enable ORC support - missing dependencies") set(USE_INTERNAL_ORC_LIBRARY 0) endif() -endif() - message (STATUS "Using internal=${USE_INTERNAL_ORC_LIBRARY} orc=${USE_ORC}: ${ORC_INCLUDE_DIR} : ${ORC_LIBRARY}") diff --git a/cmake/find/parquet.cmake b/cmake/find/parquet.cmake index d4f62b87d29..3cb6be38c3f 100644 --- a/cmake/find/parquet.cmake +++ b/cmake/find/parquet.cmake @@ -1,24 +1,39 @@ if (Protobuf_PROTOC_EXECUTABLE) option (ENABLE_PARQUET "Enable parquet" ${ENABLE_LIBRARIES}) +elseif(ENABLE_PARQUET OR USE_INTERNAL_PARQUET_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use parquet without protoc executable") endif() -if (ENABLE_PARQUET) +if (NOT ENABLE_PARQUET) + if(USE_INTERNAL_PARQUET_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal parquet with ENABLE_PARQUET=OFF") + endif() + message(STATUS "Building without Parquet support") + return() +endif() if (NOT OS_FREEBSD) # Freebsd: ../contrib/arrow/cpp/src/arrow/util/bit-util.h:27:10: fatal error: endian.h: No such file or directory option(USE_INTERNAL_PARQUET_LIBRARY "Set to FALSE to use system parquet library instead of bundled" ${NOT_UNBUNDLED}) +elseif(USE_INTERNAL_PARQUET_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Using internal parquet is not supported on freebsd") endif() if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/CMakeLists.txt") if(USE_INTERNAL_PARQUET_LIBRARY) message(WARNING "submodule contrib/arrow (required for Parquet) is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal parquet library") + set(USE_INTERNAL_PARQUET_LIBRARY 0) endif() - set(USE_INTERNAL_PARQUET_LIBRARY 0) set(MISSING_INTERNAL_PARQUET_LIBRARY 1) endif() if(NOT USE_INTERNAL_PARQUET_LIBRARY) find_package(Arrow) find_package(Parquet) + + if(NOT ARROW_INCLUDE_DIR OR PARQUET_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system parquet: arrow=${ARROW_INCLUDE_DIR} parquet=${PARQUET_INCLUDE_DIR}" ) + endif() endif() if(ARROW_INCLUDE_DIR AND PARQUET_INCLUDE_DIR) @@ -26,6 +41,8 @@ elseif(NOT MISSING_INTERNAL_PARQUET_LIBRARY AND NOT OS_FREEBSD) include(cmake/find/snappy.cmake) if(SNAPPY_LIBRARY) set(CAN_USE_INTERNAL_PARQUET_LIBRARY 1) + else() + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal parquet library without snappy") endif() include(CheckCXXSourceCompiles) @@ -33,19 +50,20 @@ elseif(NOT MISSING_INTERNAL_PARQUET_LIBRARY AND NOT OS_FREEBSD) set(CMAKE_REQUIRED_LIBRARIES ${DOUBLE_CONVERSION_LIBRARIES}) set(CMAKE_REQUIRED_INCLUDES ${DOUBLE_CONVERSION_INCLUDE_DIR}) check_cxx_source_compiles(" - #include - int main() { static const int flags_ = double_conversion::StringToDoubleConverter::ALLOW_CASE_INSENSIBILITY; return 0;} + #include + int main() { static const int flags_ = double_conversion::StringToDoubleConverter::ALLOW_CASE_INSENSIBILITY; return 0;} " HAVE_DOUBLE_CONVERSION_ALLOW_CASE_INSENSIBILITY) if(NOT HAVE_DOUBLE_CONVERSION_ALLOW_CASE_INSENSIBILITY) # HAVE_STD_RANDOM_SHUFFLE - message(STATUS "Disabling internal parquet library because arrow is broken (can't use old double_conversion)") - set(CAN_USE_INTERNAL_PARQUET_LIBRARY 0) + message (${RECONFIGURE_MESSAGE_LEVEL} "Disabling internal parquet library because arrow is broken (can't use old double_conversion)") + set(CAN_USE_INTERNAL_PARQUET_LIBRARY 0) endif() endif() - if(NOT CAN_USE_INTERNAL_PARQUET_LIBRARY) + if(NOT CAN_USE_INTERNAL_PARQUET_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal parquet") set(USE_INTERNAL_PARQUET_LIBRARY 0) - else() + else() set(USE_INTERNAL_PARQUET_LIBRARY 1) if(USE_INTERNAL_PARQUET_LIBRARY_NATIVE_CMAKE) @@ -53,7 +71,7 @@ elseif(NOT MISSING_INTERNAL_PARQUET_LIBRARY AND NOT OS_FREEBSD) set(PARQUET_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src" ${ClickHouse_BINARY_DIR}/contrib/arrow/cpp/src) endif() - if(${USE_STATIC_LIBRARIES}) + if(MAKE_STATIC_LIBRARIES) set(FLATBUFFERS_LIBRARY flatbuffers) set(ARROW_LIBRARY arrow_static) set(PARQUET_LIBRARY parquet_static) @@ -72,8 +90,8 @@ elseif(NOT MISSING_INTERNAL_PARQUET_LIBRARY AND NOT OS_FREEBSD) set(USE_ORC 1) set(USE_ARROW 1) endif() -endif() - +elseif(OS_FREEBSD) + message (${RECONFIGURE_MESSAGE_LEVEL} "Using internal parquet library on FreeBSD is not supported") endif() if(USE_PARQUET) diff --git a/cmake/find/protobuf.cmake b/cmake/find/protobuf.cmake index 51110d3b6a3..49d09299f27 100644 --- a/cmake/find/protobuf.cmake +++ b/cmake/find/protobuf.cmake @@ -1,12 +1,18 @@ option(ENABLE_PROTOBUF "Enable protobuf" ${ENABLE_LIBRARIES}) -if(ENABLE_PROTOBUF) +if(NOT ENABLE_PROTOBUF) + if(USE_INTERNAL_PROTOBUF_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal protobuf with ENABLE_PROTOBUF=OFF") + endif() + return() +endif() option(USE_INTERNAL_PROTOBUF_LIBRARY "Set to FALSE to use system protobuf instead of bundled" ${NOT_UNBUNDLED}) if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/protobuf/cmake/CMakeLists.txt") if(USE_INTERNAL_PROTOBUF_LIBRARY) message(WARNING "submodule contrib/protobuf is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal protobuf") set(USE_INTERNAL_PROTOBUF_LIBRARY 0) endif() set(MISSING_INTERNAL_PROTOBUF_LIBRARY 1) @@ -14,6 +20,9 @@ endif() if(NOT USE_INTERNAL_PROTOBUF_LIBRARY) find_package(Protobuf) + if (NOT Protobuf_LIBRARY OR NOT Protobuf_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system protobuf") + endif() endif() if (Protobuf_LIBRARY AND Protobuf_INCLUDE_DIR) @@ -36,11 +45,11 @@ if(OS_FREEBSD AND SANITIZE STREQUAL "address") if(LLVM_INCLUDE_DIRS) set(Protobuf_INCLUDE_DIR ${Protobuf_INCLUDE_DIR} ${LLVM_INCLUDE_DIRS}) else() + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use protobuf on FreeBSD with address sanitizer without LLVM") set(USE_PROTOBUF 0) endif() endif() include (${ClickHouse_SOURCE_DIR}/cmake/protobuf_generate_cpp.cmake) -endif() message(STATUS "Using protobuf=${USE_PROTOBUF}: ${Protobuf_INCLUDE_DIR} : ${Protobuf_LIBRARY} : ${Protobuf_PROTOC_EXECUTABLE}") diff --git a/cmake/find/rapidjson.cmake b/cmake/find/rapidjson.cmake index df896b644e5..f880d19143e 100644 --- a/cmake/find/rapidjson.cmake +++ b/cmake/find/rapidjson.cmake @@ -1,5 +1,8 @@ option(ENABLE_RAPIDJSON "Use rapidjson" ${ENABLE_LIBRARIES}) if(NOT ENABLE_RAPIDJSON) + if(USE_INTERNAL_RAPIDJSON_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal rapidjson library with ENABLE_RAPIDJSON=OFF") + endif() return() endif() @@ -8,6 +11,7 @@ option(USE_INTERNAL_RAPIDJSON_LIBRARY "Set to FALSE to use system rapidjson libr if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rapidjson/include/rapidjson/rapidjson.h") if(USE_INTERNAL_RAPIDJSON_LIBRARY) message(WARNING "submodule contrib/rapidjson is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal rapidjson library") set(USE_INTERNAL_RAPIDJSON_LIBRARY 0) endif() set(MISSING_INTERNAL_RAPIDJSON_LIBRARY 1) @@ -15,6 +19,9 @@ endif() if(NOT USE_INTERNAL_RAPIDJSON_LIBRARY) find_path(RAPIDJSON_INCLUDE_DIR NAMES rapidjson/rapidjson.h PATHS ${RAPIDJSON_INCLUDE_PATHS}) + if(NOT RAPIDJSON_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system rapidjson") + endif() endif() if(RAPIDJSON_INCLUDE_DIR) diff --git a/cmake/find/rdkafka.cmake b/cmake/find/rdkafka.cmake index f18674dd440..7b213fd37ea 100644 --- a/cmake/find/rdkafka.cmake +++ b/cmake/find/rdkafka.cmake @@ -1,30 +1,55 @@ # Freebsd: contrib/cppkafka/include/cppkafka/detail/endianness.h:53:23: error: 'betoh16' was not declared in this scope if (NOT ARCH_ARM AND NOT OS_FREEBSD AND OPENSSL_FOUND) option (ENABLE_RDKAFKA "Enable kafka" ${ENABLE_LIBRARIES}) +elseif(ENABLE_RDKAFKA AND NOT OPENSSL_FOUND) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use librdkafka without SSL") +else(ENABLE_RDKAFKA) + message (${RECONFIGURE_MESSAGE_LEVEL} "librdafka is not supported on ARM and on FreeBSD") endif () -if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cppkafka/CMakeLists.txt") - message (WARNING "submodule contrib/cppkafka is missing. to fix try run: \n git submodule update --init --recursive") - set (ENABLE_RDKAFKA 0) -endif () - -if (ENABLE_RDKAFKA) +if (NOT ENABLE_RDKAFKA) + if (USE_INTERNAL_RDKAFKA_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal librdkafka with ENABLE_RDKAFKA=OFF") + endif() + return() +endif() if (NOT ARCH_ARM AND USE_LIBGSASL) option (USE_INTERNAL_RDKAFKA_LIBRARY "Set to FALSE to use system librdkafka instead of the bundled" ${NOT_UNBUNDLED}) +elseif(USE_INTERNAL_RDKAFKA_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal librdkafka with ARCH_ARM=${ARCH_ARM} AND USE_LIBGSASL=${USE_LIBGSASL}") endif () -if (USE_INTERNAL_RDKAFKA_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/librdkafka/CMakeLists.txt") - message (WARNING "submodule contrib/librdkafka is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_RDKAFKA_LIBRARY 0) - set (MISSING_INTERNAL_RDKAFKA_LIBRARY 1) +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cppkafka/CMakeLists.txt") + if(USE_INTERNAL_RDKAFKA_LIBRARY) + message (WARNING "submodule contrib/cppkafka is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal cppkafka") + set (USE_INTERNAL_RDKAFKA_LIBRARY 0) + endif() + set (MISSING_INTERNAL_CPPKAFKA_LIBRARY 1) +endif () + +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/librdkafka/CMakeLists.txt") + if(USE_INTERNAL_RDKAFKA_LIBRARY OR MISSING_INTERNAL_CPPKAFKA_LIBRARY) + message (WARNING "submodule contrib/librdkafka is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal rdkafka") + set (USE_INTERNAL_RDKAFKA_LIBRARY 0) + endif() + set (MISSING_INTERNAL_RDKAFKA_LIBRARY 1) endif () if (NOT USE_INTERNAL_RDKAFKA_LIBRARY) find_library (RDKAFKA_LIB rdkafka) find_path (RDKAFKA_INCLUDE_DIR NAMES librdkafka/rdkafka.h PATHS ${RDKAFKA_INCLUDE_PATHS}) + if (NOT RDKAFKA_LIB OR NOT RDKAFKA_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system librdkafka") + endif() + if (USE_STATIC_LIBRARIES AND NOT OS_FREEBSD) find_library (SASL2_LIBRARY sasl2) + if (NOT SASL2_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system sasl2 library needed for static librdkafka") + endif() endif () set (CPPKAFKA_LIBRARY cppkafka) # TODO: try to use unbundled version. endif () @@ -39,14 +64,14 @@ if (RDKAFKA_LIB AND RDKAFKA_INCLUDE_DIR) if (LZ4_LIBRARY) list (APPEND RDKAFKA_LIBRARY ${LZ4_LIBRARY}) endif () -elseif (NOT MISSING_INTERNAL_RDKAFKA_LIBRARY AND NOT ARCH_ARM) +elseif (NOT MISSING_INTERNAL_RDKAFKA_LIBRARY AND NOT MISSING_INTERNAL_CPPKAFKA_LIBRARY AND NOT ARCH_ARM) set (USE_INTERNAL_RDKAFKA_LIBRARY 1) set (RDKAFKA_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/librdkafka/src") set (RDKAFKA_LIBRARY rdkafka) set (CPPKAFKA_LIBRARY cppkafka) set (USE_RDKAFKA 1) -endif () - +elseif(ARCH_ARM) + message (${RECONFIGURE_MESSAGE_LEVEL} "Using internal rdkafka on ARM is not supported") endif () message (STATUS "Using librdkafka=${USE_RDKAFKA}: ${RDKAFKA_INCLUDE_DIR} : ${RDKAFKA_LIBRARY} ${CPPKAFKA_LIBRARY}") diff --git a/cmake/find/re2.cmake b/cmake/find/re2.cmake index 05ba80f143f..87bc974c788 100644 --- a/cmake/find/re2.cmake +++ b/cmake/find/re2.cmake @@ -3,6 +3,7 @@ option (USE_INTERNAL_RE2_LIBRARY "Set to FALSE to use system re2 library instead if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/re2/CMakeLists.txt") if(USE_INTERNAL_RE2_LIBRARY) message(WARNING "submodule contrib/re2 is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal re2 library") endif() set(USE_INTERNAL_RE2_LIBRARY 0) set(MISSING_INTERNAL_RE2_LIBRARY 1) @@ -11,6 +12,9 @@ endif() if (NOT USE_INTERNAL_RE2_LIBRARY) find_library (RE2_LIBRARY re2) find_path (RE2_INCLUDE_DIR NAMES re2/re2.h PATHS ${RE2_INCLUDE_PATHS}) + if (NOT RE2_LIBRARY OR NOT RE2_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system re2 library") + endif () endif () string(FIND ${CMAKE_CURRENT_BINARY_DIR} " " _have_space) @@ -24,12 +28,13 @@ if (RE2_LIBRARY AND RE2_INCLUDE_DIR) elseif (NOT MISSING_INTERNAL_RE2_LIBRARY) set (USE_INTERNAL_RE2_LIBRARY 1) set (RE2_LIBRARY re2) - set (RE2_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/re2) + set (RE2_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/re2") if (NOT MISSING_INTERNAL_RE2_ST_LIBRARY) set (RE2_ST_LIBRARY re2_st) set (USE_RE2_ST 1) else () set (RE2_ST_LIBRARY ${RE2_LIBRARY}) + message (${RECONFIGURE_MESSAGE_LEVEL} "Using internal re2 library instead of re2_st") endif () endif () diff --git a/cmake/find/s3.cmake b/cmake/find/s3.cmake index af53dc80feb..bf15c4944bb 100644 --- a/cmake/find/s3.cmake +++ b/cmake/find/s3.cmake @@ -1,26 +1,39 @@ if(NOT OS_FREEBSD AND NOT APPLE AND NOT ARCH_ARM) option(ENABLE_S3 "Enable S3" ${ENABLE_LIBRARIES}) +elseif(ENABLE_S3 OR USE_INTERNAL_AWS_S3_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use S3 on ARM, Apple or FreeBSD") endif() -if(ENABLE_S3) - option(USE_INTERNAL_AWS_S3_LIBRARY "Set to FALSE to use system S3 instead of bundled" ${NOT_UNBUNDLED}) - - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-s3") - message (WARNING "submodule contrib/aws is missing. to fix try run: \n git submodule update --init --recursive") - set (MISSING_AWS_S3 1) - endif () - - if (USE_INTERNAL_AWS_S3_LIBRARY AND NOT MISSING_AWS_S3) - set(AWS_S3_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-s3/include") - set(AWS_S3_CORE_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-core/include") - set(AWS_S3_LIBRARY aws_s3) - set(USE_INTERNAL_AWS_S3_LIBRARY 1) - set(USE_AWS_S3 1) - else() - set(USE_INTERNAL_AWS_S3_LIBRARY 0) - set(USE_AWS_S3 0) - endif () - +if(NOT ENABLE_S3) + if(USE_INTERNAL_AWS_S3_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal S3 library with ENABLE_S3=OFF") + endif() + return() endif() +option(USE_INTERNAL_AWS_S3_LIBRARY "Set to FALSE to use system S3 instead of bundled" ${NOT_UNBUNDLED}) +if (NOT USE_INTERNAL_AWS_S3_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Compilation with external S3 library is not supported yet") +endif() + +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-s3") + message (WARNING "submodule contrib/aws is missing. to fix try run: \n git submodule update --init --recursive") + if (USE_INTERNAL_AWS_S3_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal S3 library") + endif () + set (MISSING_AWS_S3 1) +endif () + +if (USE_INTERNAL_AWS_S3_LIBRARY AND NOT MISSING_AWS_S3) + set(AWS_S3_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-s3/include") + set(AWS_S3_CORE_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-core/include") + set(AWS_S3_LIBRARY aws_s3) + set(USE_INTERNAL_AWS_S3_LIBRARY 1) + set(USE_AWS_S3 1) +else() + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't enable S3") + set(USE_INTERNAL_AWS_S3_LIBRARY 0) + set(USE_AWS_S3 0) +endif () + message (STATUS "Using aws_s3=${USE_AWS_S3}: ${AWS_S3_INCLUDE_DIR} : ${AWS_S3_LIBRARY}") diff --git a/cmake/find/sentry.cmake b/cmake/find/sentry.cmake index 84425220f12..2936c045f99 100644 --- a/cmake/find/sentry.cmake +++ b/cmake/find/sentry.cmake @@ -2,6 +2,9 @@ set (SENTRY_LIBRARY "sentry") set (SENTRY_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/sentry-native/include") if (NOT EXISTS "${SENTRY_INCLUDE_DIR}/sentry.h") message (WARNING "submodule contrib/sentry-native is missing. to fix try run: \n git submodule update --init --recursive") + if (USE_SENTRY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal sentry library") + endif() return() endif () @@ -16,4 +19,6 @@ if (NOT OS_FREEBSD AND NOT SPLIT_SHARED_LIBRARIES AND NOT_UNBUNDLED AND NOT (OS_ message (STATUS "Using sentry=${USE_SENTRY}: ${SENTRY_LIBRARY}") include_directories("${SENTRY_INCLUDE_DIR}") +elseif (USE_SENTRY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Sentry is not supported in current configuration") endif () diff --git a/cmake/find/simdjson.cmake b/cmake/find/simdjson.cmake index 4b1f3224076..ef0c744d4d6 100644 --- a/cmake/find/simdjson.cmake +++ b/cmake/find/simdjson.cmake @@ -1,8 +1,11 @@ +option (USE_SIMDJSON "Use simdjson" ON) + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/simdjson/include/simdjson.h") message (WARNING "submodule contrib/simdjson is missing. to fix try run: \n git submodule update --init --recursive") + if (USE_SIMDJSON) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal simdjson library") + endif() return() endif () -option (USE_SIMDJSON "Use simdjson" ON) - message(STATUS "Using simdjson=${USE_SIMDJSON}") diff --git a/cmake/find/snappy.cmake b/cmake/find/snappy.cmake index 504db87d613..e719231c338 100644 --- a/cmake/find/snappy.cmake +++ b/cmake/find/snappy.cmake @@ -1,13 +1,21 @@ option(USE_SNAPPY "Enable support of snappy library" ${ENABLE_LIBRARIES}) -if(USE_SNAPPY) - option (USE_INTERNAL_SNAPPY_LIBRARY "Set to FALSE to use system snappy library instead of bundled" ${NOT_UNBUNDLED}) - - if(NOT USE_INTERNAL_SNAPPY_LIBRARY) - find_library(SNAPPY_LIBRARY snappy) - else () - set(SNAPPY_LIBRARY snappy) +if(NOT USE_SNAPPY) + if (USE_INTERNAL_SNAPPY_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal snappy library with USE_SNAPPY=OFF") endif() + return() +endif() - message (STATUS "Using snappy: ${SNAPPY_LIBRARY}") -endif () +option (USE_INTERNAL_SNAPPY_LIBRARY "Set to FALSE to use system snappy library instead of bundled" ${NOT_UNBUNDLED}) + +if(NOT USE_INTERNAL_SNAPPY_LIBRARY) + find_library(SNAPPY_LIBRARY snappy) + if (NOT SNAPPY_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system snappy library") + endif() +else () + set(SNAPPY_LIBRARY snappy) +endif() + +message (STATUS "Using snappy: ${SNAPPY_LIBRARY}") diff --git a/cmake/find/sparsehash.cmake b/cmake/find/sparsehash.cmake index d34ed8e048b..f8b6d293705 100644 --- a/cmake/find/sparsehash.cmake +++ b/cmake/find/sparsehash.cmake @@ -2,6 +2,9 @@ option (USE_INTERNAL_SPARSEHASH_LIBRARY "Set to FALSE to use system sparsehash l if (NOT USE_INTERNAL_SPARSEHASH_LIBRARY) find_path (SPARSEHASH_INCLUDE_DIR NAMES sparsehash/sparse_hash_map PATHS ${SPARSEHASH_INCLUDE_PATHS}) + if (NOT SPARSEHASH_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system sparsehash library") + endif () endif () if (SPARSEHASH_INCLUDE_DIR) diff --git a/cmake/find/ssl.cmake b/cmake/find/ssl.cmake index efc9127309c..9058857c173 100644 --- a/cmake/find/ssl.cmake +++ b/cmake/find/ssl.cmake @@ -1,12 +1,18 @@ option(ENABLE_SSL "Enable ssl" ${ENABLE_LIBRARIES}) -if(ENABLE_SSL) +if(NOT ENABLE_SSL) + if (USE_INTERNAL_SSL_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal ssl library with ENABLE_SSL=OFF") + endif() + return() +endif() option(USE_INTERNAL_SSL_LIBRARY "Set to FALSE to use system *ssl library instead of bundled" ${NOT_UNBUNDLED}) if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/openssl/README") if(USE_INTERNAL_SSL_LIBRARY) message(WARNING "submodule contrib/openssl is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal ssl library") endif() set(USE_INTERNAL_SSL_LIBRARY 0) set(MISSING_INTERNAL_SSL_LIBRARY 1) @@ -36,6 +42,10 @@ if (NOT USE_INTERNAL_SSL_LIBRARY) set (OPENSSL_FOUND 1) endif () endif () + + if (NOT OPENSSL_FOUND) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system ssl") + endif() endif () if (NOT OPENSSL_FOUND AND NOT MISSING_INTERNAL_SSL_LIBRARY) @@ -123,7 +133,4 @@ if(OPENSSL_FOUND AND NOT USE_INTERNAL_SSL_LIBRARY) endif() endif() - -endif () - message (STATUS "Using ssl=${USE_SSL}: ${OPENSSL_INCLUDE_DIR} : ${OPENSSL_LIBRARIES}") diff --git a/cmake/find/stats.cmake b/cmake/find/stats.cmake index ef5b1a73659..339e8524598 100644 --- a/cmake/find/stats.cmake +++ b/cmake/find/stats.cmake @@ -14,6 +14,10 @@ if (ENABLE_STATS) set(GCEM_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/gcem/include) set (USE_STATS 1) endif() + + if (NOT USE_STATS) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't enable stats library") + endif() endif() message (STATUS "Using stats=${USE_STATS} : ${STATS_INCLUDE_DIR}") diff --git a/cmake/find/termcap.cmake b/cmake/find/termcap.cmake index 7564b7134e7..80011cd825c 100644 --- a/cmake/find/termcap.cmake +++ b/cmake/find/termcap.cmake @@ -5,4 +5,7 @@ endif() if (NOT TERMCAP_LIBRARY) find_library (TERMCAP_LIBRARY termcap) endif() +if (NOT TERMCAP_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find tinfo, ncurses or termcap library") +endif() message (STATUS "Using termcap: ${TERMCAP_LIBRARY}") diff --git a/cmake/find/zlib.cmake b/cmake/find/zlib.cmake index f65d379f577..754cd8c85bb 100644 --- a/cmake/find/zlib.cmake +++ b/cmake/find/zlib.cmake @@ -6,12 +6,14 @@ else () set (INTERNAL_ZLIB_NAME "zlib" CACHE INTERNAL "") if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}") message (WARNING "Will use standard zlib, please clone manually:\n git clone https://github.com/madler/zlib.git ${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal zlib library") endif () endif () if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}/zlib.h") if(USE_INTERNAL_ZLIB_LIBRARY) message(WARNING "submodule contrib/${INTERNAL_ZLIB_NAME} is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal zlib library") endif() set(USE_INTERNAL_ZLIB_LIBRARY 0) set(MISSING_INTERNAL_ZLIB_LIBRARY 1) @@ -19,6 +21,7 @@ endif() if (NOT USE_INTERNAL_ZLIB_LIBRARY) find_package (ZLIB) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system zlib library") endif () if (NOT ZLIB_FOUND AND NOT MISSING_INTERNAL_ZLIB_LIBRARY) diff --git a/cmake/find/zstd.cmake b/cmake/find/zstd.cmake index e4f32d4b170..b12bb701e0e 100644 --- a/cmake/find/zstd.cmake +++ b/cmake/find/zstd.cmake @@ -3,14 +3,18 @@ option (USE_INTERNAL_ZSTD_LIBRARY "Set to FALSE to use system zstd library inste if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/zstd/lib/zstd.h") if(USE_INTERNAL_ZSTD_LIBRARY) message(WARNING "submodule contrib/zstd is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal zstd library") + set(USE_INTERNAL_ZSTD_LIBRARY 0) endif() - set(USE_INTERNAL_ZSTD_LIBRARY 0) set(MISSING_INTERNAL_ZSTD_LIBRARY 1) endif() if (NOT USE_INTERNAL_ZSTD_LIBRARY) find_library (ZSTD_LIBRARY zstd) find_path (ZSTD_INCLUDE_DIR NAMES zstd.h PATHS ${ZSTD_INCLUDE_PATHS}) + if (NOT ZSTD_LIBRARY OR NOT ZSTD_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system zstd library") + endif () endif () if (ZSTD_LIBRARY AND ZSTD_INCLUDE_DIR) diff --git a/cmake/tools.cmake b/cmake/tools.cmake index 95e00ad9951..a6ea573a59d 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -41,25 +41,23 @@ STRING(REGEX MATCHALL "[0-9]+" COMPILER_VERSION_LIST ${CMAKE_CXX_COMPILER_VERSIO LIST(GET COMPILER_VERSION_LIST 0 COMPILER_VERSION_MAJOR) option (LINKER_NAME "Linker name or full path") -if (COMPILER_GCC) +if (COMPILER_GCC AND NOT LINKER_NAME) find_program (LLD_PATH NAMES "ld.lld") find_program (GOLD_PATH NAMES "ld.gold") -else () +elseif (NOT LINKER_NAME) find_program (LLD_PATH NAMES "ld.lld-${COMPILER_VERSION_MAJOR}" "lld-${COMPILER_VERSION_MAJOR}" "ld.lld" "lld") find_program (GOLD_PATH NAMES "ld.gold" "gold") endif () -if (OS_LINUX) +if (OS_LINUX AND NOT LINKER_NAME) # We prefer LLD linker over Gold or BFD on Linux. - if (NOT LINKER_NAME) - if (LLD_PATH) - if (COMPILER_GCC) - # GCC driver requires one of supported linker names like "lld". - set (LINKER_NAME "lld") - else () - # Clang driver simply allows full linker path. - set (LINKER_NAME ${LLD_PATH}) - endif () + if (LLD_PATH) + if (COMPILER_GCC) + # GCC driver requires one of supported linker names like "lld". + set (LINKER_NAME "lld") + else () + # Clang driver simply allows full linker path. + set (LINKER_NAME ${LLD_PATH}) endif () endif () From d22de6c7e92c33a8ba28fec133bacf1c7959be75 Mon Sep 17 00:00:00 2001 From: Konstantin Podshumok Date: Sun, 9 Aug 2020 08:36:31 +0300 Subject: [PATCH 208/374] cmake: Make sure USE_SIMDJSON default value is in sync with ENABLE_LIBRARIES Signed-off-by: Konstantin Podshumok --- cmake/find/simdjson.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/find/simdjson.cmake b/cmake/find/simdjson.cmake index ef0c744d4d6..cffe20bdb2d 100644 --- a/cmake/find/simdjson.cmake +++ b/cmake/find/simdjson.cmake @@ -1,4 +1,4 @@ -option (USE_SIMDJSON "Use simdjson" ON) +option (USE_SIMDJSON "Use simdjson" ${ENABLE_LIBRARIES}) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/simdjson/include/simdjson.h") message (WARNING "submodule contrib/simdjson is missing. to fix try run: \n git submodule update --init --recursive") From 262080132be21ed34972de7a02bbd8d03e683e89 Mon Sep 17 00:00:00 2001 From: Konstantin Podshumok Date: Sun, 9 Aug 2020 21:31:00 +0300 Subject: [PATCH 209/374] cmake: disabling silent reconfigurations in contrib/*/CMakeLists.txt Signed-off-by: Konstantin Podshumok --- contrib/arrow-cmake/CMakeLists.txt | 2 +- contrib/boost-cmake/CMakeLists.txt | 82 ++-- contrib/cctz-cmake/CMakeLists.txt | 64 +-- contrib/curl-cmake/CMakeLists.txt | 363 +++++++------- contrib/hyperscan-cmake/CMakeLists.txt | 494 +++++++++---------- contrib/jemalloc-cmake/CMakeLists.txt | 293 ++++++------ contrib/libcpuid-cmake/CMakeLists.txt | 55 ++- contrib/libhdfs3-cmake/CMakeLists.txt | 12 +- contrib/lz4-cmake/CMakeLists.txt | 37 +- contrib/poco-cmake/CMakeLists.txt | 3 + contrib/replxx-cmake/CMakeLists.txt | 88 ++-- contrib/unixodbc-cmake/CMakeLists.txt | 628 +++++++++++++------------ 12 files changed, 1126 insertions(+), 995 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index be8cb9a9708..442f2534f6a 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -80,7 +80,7 @@ set(FLATBUFFERS_BINARY_DIR ${ClickHouse_BINARY_DIR}/contrib/flatbuffers) set(FLATBUFFERS_INCLUDE_DIR ${FLATBUFFERS_SRC_DIR}/include) # set flatbuffers CMake options -if (${USE_STATIC_LIBRARIES}) +if (MAKE_STATIC_LIBRARIES) set(FLATBUFFERS_BUILD_FLATLIB ON CACHE BOOL "Enable the build of the flatbuffers library") set(FLATBUFFERS_BUILD_SHAREDLIB OFF CACHE BOOL "Disable the build of the flatbuffers shared library") else () diff --git a/contrib/boost-cmake/CMakeLists.txt b/contrib/boost-cmake/CMakeLists.txt index e92fe4b7159..4d6a2ff0e62 100644 --- a/contrib/boost-cmake/CMakeLists.txt +++ b/contrib/boost-cmake/CMakeLists.txt @@ -1,6 +1,52 @@ option (USE_INTERNAL_BOOST_LIBRARY "Use internal Boost library" ${NOT_UNBUNDLED}) -if (USE_INTERNAL_BOOST_LIBRARY) +if (NOT USE_INTERNAL_BOOST_LIBRARY) + # 1.70 like in contrib/boost + # 1.67 on CI + set(BOOST_VERSION 1.67) + + find_package(Boost ${BOOST_VERSION} COMPONENTS + system + filesystem + iostreams + program_options + regex + ) + + if(Boost_INCLUDE_DIR AND Boost_FILESYSTEM_LIBRARY AND Boost_FILESYSTEM_LIBRARY AND + Boost_PROGRAM_OPTIONS_LIBRARY AND Boost_REGEX_LIBRARY AND Boost_SYSTEM_LIBRARY) + + set(EXTERNAL_BOOST_FOUND 1) + + add_library (_boost_headers_only INTERFACE) + add_library (boost::headers_only ALIAS _boost_headers_only) + target_include_directories (_boost_headers_only SYSTEM BEFORE INTERFACE ${Boost_INCLUDE_DIR}) + + add_library (_boost_filesystem INTERFACE) + add_library (_boost_iostreams INTERFACE) + add_library (_boost_program_options INTERFACE) + add_library (_boost_regex INTERFACE) + add_library (_boost_system INTERFACE) + + target_link_libraries (_boost_filesystem INTERFACE ${Boost_FILESYSTEM_LIBRARY}) + target_link_libraries (_boost_iostreams INTERFACE ${Boost_IOSTREAMS_LIBRARY}) + target_link_libraries (_boost_program_options INTERFACE ${Boost_PROGRAM_OPTIONS_LIBRARY}) + target_link_libraries (_boost_regex INTERFACE ${Boost_REGEX_LIBRARY}) + target_link_libraries (_boost_system INTERFACE ${Boost_SYSTEM_LIBRARY}) + + add_library (boost::filesystem ALIAS _boost_filesystem) + add_library (boost::iostreams ALIAS _boost_iostreams) + add_library (boost::program_options ALIAS _boost_program_options) + add_library (boost::regex ALIAS _boost_regex) + add_library (boost::system ALIAS _boost_system) + else() + set(EXTERNAL_BOOST_FOUND 0) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system boost") + endif() +endif() + +if (NOT EXTERNAL_BOOST_FOUND) + set (USE_INTERNAL_BOOST_LIBRARY 1) set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/boost) # filesystem @@ -96,38 +142,4 @@ if (USE_INTERNAL_BOOST_LIBRARY) add_library (_boost_system ${SRCS_SYSTEM}) add_library (boost::system ALIAS _boost_system) target_include_directories (_boost_system PRIVATE ${LIBRARY_DIR}) -else () - # 1.70 like in contrib/boost - # 1.67 on CI - set(BOOST_VERSION 1.67) - - find_package(Boost ${BOOST_VERSION} COMPONENTS - system - filesystem - iostreams - program_options - regex - REQUIRED) - - add_library (_boost_headers_only INTERFACE) - add_library (boost::headers_only ALIAS _boost_headers_only) - target_include_directories (_boost_headers_only SYSTEM BEFORE INTERFACE ${Boost_INCLUDE_DIR}) - - add_library (_boost_filesystem INTERFACE) - add_library (_boost_iostreams INTERFACE) - add_library (_boost_program_options INTERFACE) - add_library (_boost_regex INTERFACE) - add_library (_boost_system INTERFACE) - - target_link_libraries (_boost_filesystem INTERFACE ${Boost_FILESYSTEM_LIBRARY}) - target_link_libraries (_boost_iostreams INTERFACE ${Boost_IOSTREAMS_LIBRARY}) - target_link_libraries (_boost_program_options INTERFACE ${Boost_PROGRAM_OPTIONS_LIBRARY}) - target_link_libraries (_boost_regex INTERFACE ${Boost_REGEX_LIBRARY}) - target_link_libraries (_boost_system INTERFACE ${Boost_SYSTEM_LIBRARY}) - - add_library (boost::filesystem ALIAS _boost_filesystem) - add_library (boost::iostreams ALIAS _boost_iostreams) - add_library (boost::program_options ALIAS _boost_program_options) - add_library (boost::regex ALIAS _boost_regex) - add_library (boost::system ALIAS _boost_system) endif () diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 7eb768b8de9..5f6768a3f4f 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -1,9 +1,42 @@ -option (USE_INTERNAL_CCTZ "Use internal cctz library" ${NOT_UNBUNDLED}) +option (USE_INTERNAL_CCTZ_LIBRARY "Use internal cctz library" ${NOT_UNBUNDLED}) -if (USE_INTERNAL_CCTZ) - SET(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/cctz) +if (NOT USE_INTERNAL_CCTZ_LIBRARY) + find_library (LIBRARY_CCTZ cctz) + find_path (INCLUDE_CCTZ NAMES cctz/civil_time.h) - SET (SRCS + if (LIBRARY_CCTZ AND INCLUDE_CCTZ) + set (EXTERNAL_CCTZ_LIBRARY_FOUND 1) + + set(CMAKE_REQUIRED_LIBRARIES ${LIBRARY_CCTZ}) + set(CMAKE_REQUIRED_INCLUDES ${INCLUDE_CCTZ}) + check_cxx_source_compiles( + " + #include + int main() { + cctz::civil_day date; + } + " + EXTERNAL_CCTZ_LIBRARY_WORKS + ) + + if (NOT EXTERNAL_CCTZ_LIBRARY_WORKS) + message (${RECONFIGURE_MESSAGE_LEVEL} "External cctz is not working: ${LIBRARY_CCTZ} ${INCLUDE_CCTZ}") + else() + add_library (cctz UNKNOWN IMPORTED) + set_property (TARGET cctz PROPERTY IMPORTED_LOCATION ${LIBRARY_CCTZ}) + set_property (TARGET cctz PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_CCTZ}) + endif() + else() + set (EXTERNAL_CCTZ_LIBRARY_FOUND 0) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system cctz") + endif() +endif() + +if (NOT EXTERNAL_CCTZ_LIBRARY_FOUND OR NOT EXTERNAL_CCTZ_LIBRARY_WORKS) + set(USE_INTERNAL_CCTZ_LIBRARY 1) + set(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/cctz) + + set (SRCS ${LIBRARY_DIR}/src/civil_time_detail.cc ${LIBRARY_DIR}/src/time_zone_fixed.cc ${LIBRARY_DIR}/src/time_zone_format.cc @@ -618,29 +651,6 @@ if (USE_INTERNAL_CCTZ) add_dependencies(cctz tzdata) target_link_libraries(cctz INTERFACE "-Wl,--whole-archive $ -Wl,--no-whole-archive") endif () - -else () - find_library (LIBRARY_CCTZ cctz) - find_path (INCLUDE_CCTZ NAMES cctz/civil_time.h) - - add_library (cctz UNKNOWN IMPORTED) - set_property (TARGET cctz PROPERTY IMPORTED_LOCATION ${LIBRARY_CCTZ}) - set_property (TARGET cctz PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_CCTZ}) - - set(CMAKE_REQUIRED_LIBRARIES cctz) - check_cxx_source_compiles( - " - #include - int main() { - cctz::civil_day date; - } - " - EXTERNAL_CCTZ_WORKS - ) - - if (NOT EXTERNAL_CCTZ_WORKS) - message (FATAL_ERROR "cctz is unusable: ${LIBRARY_CCTZ} ${INCLUDE_CCTZ}") - endif () endif () message (STATUS "Using cctz") diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index 3c3226cae9e..eb97cfa5f14 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -1,187 +1,200 @@ option (ENABLE_CURL "Enable curl" ${ENABLE_LIBRARIES}) -if (ENABLE_CURL) - option (USE_INTERNAL_CURL "Use internal curl library" ${NOT_UNBUNDLED}) - +if (NOT ENABLE_CURL) if (USE_INTERNAL_CURL) - set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/curl") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal curl with ENABLE_CURL=OFF") + endif() + return() +endif() - set (SRCS - ${LIBRARY_DIR}/lib/file.c - ${LIBRARY_DIR}/lib/timeval.c - ${LIBRARY_DIR}/lib/base64.c - ${LIBRARY_DIR}/lib/hostip.c - ${LIBRARY_DIR}/lib/progress.c - ${LIBRARY_DIR}/lib/formdata.c - ${LIBRARY_DIR}/lib/cookie.c - ${LIBRARY_DIR}/lib/http.c - ${LIBRARY_DIR}/lib/sendf.c - ${LIBRARY_DIR}/lib/url.c - ${LIBRARY_DIR}/lib/dict.c - ${LIBRARY_DIR}/lib/if2ip.c - ${LIBRARY_DIR}/lib/speedcheck.c - ${LIBRARY_DIR}/lib/ldap.c - ${LIBRARY_DIR}/lib/version.c - ${LIBRARY_DIR}/lib/getenv.c - ${LIBRARY_DIR}/lib/escape.c - ${LIBRARY_DIR}/lib/mprintf.c - ${LIBRARY_DIR}/lib/telnet.c - ${LIBRARY_DIR}/lib/netrc.c - ${LIBRARY_DIR}/lib/getinfo.c - ${LIBRARY_DIR}/lib/transfer.c - ${LIBRARY_DIR}/lib/strcase.c - ${LIBRARY_DIR}/lib/easy.c - ${LIBRARY_DIR}/lib/security.c - ${LIBRARY_DIR}/lib/curl_fnmatch.c - ${LIBRARY_DIR}/lib/fileinfo.c - ${LIBRARY_DIR}/lib/wildcard.c - ${LIBRARY_DIR}/lib/krb5.c - ${LIBRARY_DIR}/lib/memdebug.c - ${LIBRARY_DIR}/lib/http_chunks.c - ${LIBRARY_DIR}/lib/strtok.c - ${LIBRARY_DIR}/lib/connect.c - ${LIBRARY_DIR}/lib/llist.c - ${LIBRARY_DIR}/lib/hash.c - ${LIBRARY_DIR}/lib/multi.c - ${LIBRARY_DIR}/lib/content_encoding.c - ${LIBRARY_DIR}/lib/share.c - ${LIBRARY_DIR}/lib/http_digest.c - ${LIBRARY_DIR}/lib/md4.c - ${LIBRARY_DIR}/lib/md5.c - ${LIBRARY_DIR}/lib/http_negotiate.c - ${LIBRARY_DIR}/lib/inet_pton.c - ${LIBRARY_DIR}/lib/strtoofft.c - ${LIBRARY_DIR}/lib/strerror.c - ${LIBRARY_DIR}/lib/amigaos.c - ${LIBRARY_DIR}/lib/hostasyn.c - ${LIBRARY_DIR}/lib/hostip4.c - ${LIBRARY_DIR}/lib/hostip6.c - ${LIBRARY_DIR}/lib/hostsyn.c - ${LIBRARY_DIR}/lib/inet_ntop.c - ${LIBRARY_DIR}/lib/parsedate.c - ${LIBRARY_DIR}/lib/select.c - ${LIBRARY_DIR}/lib/splay.c - ${LIBRARY_DIR}/lib/strdup.c - ${LIBRARY_DIR}/lib/socks.c - ${LIBRARY_DIR}/lib/curl_addrinfo.c - ${LIBRARY_DIR}/lib/socks_gssapi.c - ${LIBRARY_DIR}/lib/socks_sspi.c - ${LIBRARY_DIR}/lib/curl_sspi.c - ${LIBRARY_DIR}/lib/slist.c - ${LIBRARY_DIR}/lib/nonblock.c - ${LIBRARY_DIR}/lib/curl_memrchr.c - ${LIBRARY_DIR}/lib/imap.c - ${LIBRARY_DIR}/lib/pop3.c - ${LIBRARY_DIR}/lib/smtp.c - ${LIBRARY_DIR}/lib/pingpong.c - ${LIBRARY_DIR}/lib/rtsp.c - ${LIBRARY_DIR}/lib/curl_threads.c - ${LIBRARY_DIR}/lib/warnless.c - ${LIBRARY_DIR}/lib/hmac.c - ${LIBRARY_DIR}/lib/curl_rtmp.c - ${LIBRARY_DIR}/lib/openldap.c - ${LIBRARY_DIR}/lib/curl_gethostname.c - ${LIBRARY_DIR}/lib/gopher.c - ${LIBRARY_DIR}/lib/idn_win32.c - ${LIBRARY_DIR}/lib/http_proxy.c - ${LIBRARY_DIR}/lib/non-ascii.c - ${LIBRARY_DIR}/lib/asyn-thread.c - ${LIBRARY_DIR}/lib/curl_gssapi.c - ${LIBRARY_DIR}/lib/http_ntlm.c - ${LIBRARY_DIR}/lib/curl_ntlm_wb.c - ${LIBRARY_DIR}/lib/curl_ntlm_core.c - ${LIBRARY_DIR}/lib/curl_sasl.c - ${LIBRARY_DIR}/lib/rand.c - ${LIBRARY_DIR}/lib/curl_multibyte.c - ${LIBRARY_DIR}/lib/hostcheck.c - ${LIBRARY_DIR}/lib/conncache.c - ${LIBRARY_DIR}/lib/dotdot.c - ${LIBRARY_DIR}/lib/x509asn1.c - ${LIBRARY_DIR}/lib/http2.c - ${LIBRARY_DIR}/lib/smb.c - ${LIBRARY_DIR}/lib/curl_endian.c - ${LIBRARY_DIR}/lib/curl_des.c - ${LIBRARY_DIR}/lib/system_win32.c - ${LIBRARY_DIR}/lib/mime.c - ${LIBRARY_DIR}/lib/sha256.c - ${LIBRARY_DIR}/lib/setopt.c - ${LIBRARY_DIR}/lib/curl_path.c - ${LIBRARY_DIR}/lib/curl_ctype.c - ${LIBRARY_DIR}/lib/curl_range.c - ${LIBRARY_DIR}/lib/psl.c - ${LIBRARY_DIR}/lib/doh.c - ${LIBRARY_DIR}/lib/urlapi.c - ${LIBRARY_DIR}/lib/curl_get_line.c - ${LIBRARY_DIR}/lib/altsvc.c - ${LIBRARY_DIR}/lib/socketpair.c - ${LIBRARY_DIR}/lib/vauth/vauth.c - ${LIBRARY_DIR}/lib/vauth/cleartext.c - ${LIBRARY_DIR}/lib/vauth/cram.c - ${LIBRARY_DIR}/lib/vauth/digest.c - ${LIBRARY_DIR}/lib/vauth/digest_sspi.c - ${LIBRARY_DIR}/lib/vauth/krb5_gssapi.c - ${LIBRARY_DIR}/lib/vauth/krb5_sspi.c - ${LIBRARY_DIR}/lib/vauth/ntlm.c - ${LIBRARY_DIR}/lib/vauth/ntlm_sspi.c - ${LIBRARY_DIR}/lib/vauth/oauth2.c - ${LIBRARY_DIR}/lib/vauth/spnego_gssapi.c - ${LIBRARY_DIR}/lib/vauth/spnego_sspi.c - ${LIBRARY_DIR}/lib/vtls/openssl.c - ${LIBRARY_DIR}/lib/vtls/gtls.c - ${LIBRARY_DIR}/lib/vtls/vtls.c - ${LIBRARY_DIR}/lib/vtls/nss.c - ${LIBRARY_DIR}/lib/vtls/polarssl.c - ${LIBRARY_DIR}/lib/vtls/polarssl_threadlock.c - ${LIBRARY_DIR}/lib/vtls/wolfssl.c - ${LIBRARY_DIR}/lib/vtls/schannel.c - ${LIBRARY_DIR}/lib/vtls/schannel_verify.c - ${LIBRARY_DIR}/lib/vtls/sectransp.c - ${LIBRARY_DIR}/lib/vtls/gskit.c - ${LIBRARY_DIR}/lib/vtls/mbedtls.c - ${LIBRARY_DIR}/lib/vtls/mesalink.c - ${LIBRARY_DIR}/lib/vtls/bearssl.c - ${LIBRARY_DIR}/lib/vquic/ngtcp2.c - ${LIBRARY_DIR}/lib/vquic/quiche.c - ${LIBRARY_DIR}/lib/vssh/libssh2.c - ${LIBRARY_DIR}/lib/vssh/libssh.c - ) +option (USE_INTERNAL_CURL "Use internal curl library" ${NOT_UNBUNDLED}) - add_library (curl ${SRCS}) +if (NOT USE_INTERNAL_CURL) + find_package (CURL) + if (NOT CURL_FOUND) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system curl") + endif() +endif() - target_compile_definitions (curl PRIVATE - HAVE_CONFIG_H - BUILDING_LIBCURL - CURL_HIDDEN_SYMBOLS - libcurl_EXPORTS - OS="${CMAKE_SYSTEM_NAME}" - ) - target_include_directories (curl PUBLIC - ${LIBRARY_DIR}/include - ${LIBRARY_DIR}/lib - . # curl_config.h - ) +if (NOT CURL_FOUND) + set (USE_INTERNAL_CURL 1) + set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/curl") - target_link_libraries (curl PRIVATE ssl) + set (SRCS + ${LIBRARY_DIR}/lib/file.c + ${LIBRARY_DIR}/lib/timeval.c + ${LIBRARY_DIR}/lib/base64.c + ${LIBRARY_DIR}/lib/hostip.c + ${LIBRARY_DIR}/lib/progress.c + ${LIBRARY_DIR}/lib/formdata.c + ${LIBRARY_DIR}/lib/cookie.c + ${LIBRARY_DIR}/lib/http.c + ${LIBRARY_DIR}/lib/sendf.c + ${LIBRARY_DIR}/lib/url.c + ${LIBRARY_DIR}/lib/dict.c + ${LIBRARY_DIR}/lib/if2ip.c + ${LIBRARY_DIR}/lib/speedcheck.c + ${LIBRARY_DIR}/lib/ldap.c + ${LIBRARY_DIR}/lib/version.c + ${LIBRARY_DIR}/lib/getenv.c + ${LIBRARY_DIR}/lib/escape.c + ${LIBRARY_DIR}/lib/mprintf.c + ${LIBRARY_DIR}/lib/telnet.c + ${LIBRARY_DIR}/lib/netrc.c + ${LIBRARY_DIR}/lib/getinfo.c + ${LIBRARY_DIR}/lib/transfer.c + ${LIBRARY_DIR}/lib/strcase.c + ${LIBRARY_DIR}/lib/easy.c + ${LIBRARY_DIR}/lib/security.c + ${LIBRARY_DIR}/lib/curl_fnmatch.c + ${LIBRARY_DIR}/lib/fileinfo.c + ${LIBRARY_DIR}/lib/wildcard.c + ${LIBRARY_DIR}/lib/krb5.c + ${LIBRARY_DIR}/lib/memdebug.c + ${LIBRARY_DIR}/lib/http_chunks.c + ${LIBRARY_DIR}/lib/strtok.c + ${LIBRARY_DIR}/lib/connect.c + ${LIBRARY_DIR}/lib/llist.c + ${LIBRARY_DIR}/lib/hash.c + ${LIBRARY_DIR}/lib/multi.c + ${LIBRARY_DIR}/lib/content_encoding.c + ${LIBRARY_DIR}/lib/share.c + ${LIBRARY_DIR}/lib/http_digest.c + ${LIBRARY_DIR}/lib/md4.c + ${LIBRARY_DIR}/lib/md5.c + ${LIBRARY_DIR}/lib/http_negotiate.c + ${LIBRARY_DIR}/lib/inet_pton.c + ${LIBRARY_DIR}/lib/strtoofft.c + ${LIBRARY_DIR}/lib/strerror.c + ${LIBRARY_DIR}/lib/amigaos.c + ${LIBRARY_DIR}/lib/hostasyn.c + ${LIBRARY_DIR}/lib/hostip4.c + ${LIBRARY_DIR}/lib/hostip6.c + ${LIBRARY_DIR}/lib/hostsyn.c + ${LIBRARY_DIR}/lib/inet_ntop.c + ${LIBRARY_DIR}/lib/parsedate.c + ${LIBRARY_DIR}/lib/select.c + ${LIBRARY_DIR}/lib/splay.c + ${LIBRARY_DIR}/lib/strdup.c + ${LIBRARY_DIR}/lib/socks.c + ${LIBRARY_DIR}/lib/curl_addrinfo.c + ${LIBRARY_DIR}/lib/socks_gssapi.c + ${LIBRARY_DIR}/lib/socks_sspi.c + ${LIBRARY_DIR}/lib/curl_sspi.c + ${LIBRARY_DIR}/lib/slist.c + ${LIBRARY_DIR}/lib/nonblock.c + ${LIBRARY_DIR}/lib/curl_memrchr.c + ${LIBRARY_DIR}/lib/imap.c + ${LIBRARY_DIR}/lib/pop3.c + ${LIBRARY_DIR}/lib/smtp.c + ${LIBRARY_DIR}/lib/pingpong.c + ${LIBRARY_DIR}/lib/rtsp.c + ${LIBRARY_DIR}/lib/curl_threads.c + ${LIBRARY_DIR}/lib/warnless.c + ${LIBRARY_DIR}/lib/hmac.c + ${LIBRARY_DIR}/lib/curl_rtmp.c + ${LIBRARY_DIR}/lib/openldap.c + ${LIBRARY_DIR}/lib/curl_gethostname.c + ${LIBRARY_DIR}/lib/gopher.c + ${LIBRARY_DIR}/lib/idn_win32.c + ${LIBRARY_DIR}/lib/http_proxy.c + ${LIBRARY_DIR}/lib/non-ascii.c + ${LIBRARY_DIR}/lib/asyn-thread.c + ${LIBRARY_DIR}/lib/curl_gssapi.c + ${LIBRARY_DIR}/lib/http_ntlm.c + ${LIBRARY_DIR}/lib/curl_ntlm_wb.c + ${LIBRARY_DIR}/lib/curl_ntlm_core.c + ${LIBRARY_DIR}/lib/curl_sasl.c + ${LIBRARY_DIR}/lib/rand.c + ${LIBRARY_DIR}/lib/curl_multibyte.c + ${LIBRARY_DIR}/lib/hostcheck.c + ${LIBRARY_DIR}/lib/conncache.c + ${LIBRARY_DIR}/lib/dotdot.c + ${LIBRARY_DIR}/lib/x509asn1.c + ${LIBRARY_DIR}/lib/http2.c + ${LIBRARY_DIR}/lib/smb.c + ${LIBRARY_DIR}/lib/curl_endian.c + ${LIBRARY_DIR}/lib/curl_des.c + ${LIBRARY_DIR}/lib/system_win32.c + ${LIBRARY_DIR}/lib/mime.c + ${LIBRARY_DIR}/lib/sha256.c + ${LIBRARY_DIR}/lib/setopt.c + ${LIBRARY_DIR}/lib/curl_path.c + ${LIBRARY_DIR}/lib/curl_ctype.c + ${LIBRARY_DIR}/lib/curl_range.c + ${LIBRARY_DIR}/lib/psl.c + ${LIBRARY_DIR}/lib/doh.c + ${LIBRARY_DIR}/lib/urlapi.c + ${LIBRARY_DIR}/lib/curl_get_line.c + ${LIBRARY_DIR}/lib/altsvc.c + ${LIBRARY_DIR}/lib/socketpair.c + ${LIBRARY_DIR}/lib/vauth/vauth.c + ${LIBRARY_DIR}/lib/vauth/cleartext.c + ${LIBRARY_DIR}/lib/vauth/cram.c + ${LIBRARY_DIR}/lib/vauth/digest.c + ${LIBRARY_DIR}/lib/vauth/digest_sspi.c + ${LIBRARY_DIR}/lib/vauth/krb5_gssapi.c + ${LIBRARY_DIR}/lib/vauth/krb5_sspi.c + ${LIBRARY_DIR}/lib/vauth/ntlm.c + ${LIBRARY_DIR}/lib/vauth/ntlm_sspi.c + ${LIBRARY_DIR}/lib/vauth/oauth2.c + ${LIBRARY_DIR}/lib/vauth/spnego_gssapi.c + ${LIBRARY_DIR}/lib/vauth/spnego_sspi.c + ${LIBRARY_DIR}/lib/vtls/openssl.c + ${LIBRARY_DIR}/lib/vtls/gtls.c + ${LIBRARY_DIR}/lib/vtls/vtls.c + ${LIBRARY_DIR}/lib/vtls/nss.c + ${LIBRARY_DIR}/lib/vtls/polarssl.c + ${LIBRARY_DIR}/lib/vtls/polarssl_threadlock.c + ${LIBRARY_DIR}/lib/vtls/wolfssl.c + ${LIBRARY_DIR}/lib/vtls/schannel.c + ${LIBRARY_DIR}/lib/vtls/schannel_verify.c + ${LIBRARY_DIR}/lib/vtls/sectransp.c + ${LIBRARY_DIR}/lib/vtls/gskit.c + ${LIBRARY_DIR}/lib/vtls/mbedtls.c + ${LIBRARY_DIR}/lib/vtls/mesalink.c + ${LIBRARY_DIR}/lib/vtls/bearssl.c + ${LIBRARY_DIR}/lib/vquic/ngtcp2.c + ${LIBRARY_DIR}/lib/vquic/quiche.c + ${LIBRARY_DIR}/lib/vssh/libssh2.c + ${LIBRARY_DIR}/lib/vssh/libssh.c + ) - # The library is large - avoid bloat (XXX: is it?) - target_compile_options (curl PRIVATE -g0) + add_library (curl ${SRCS}) - # find_package(CURL) compatibility for the following packages that uses - # find_package(CURL)/include(FindCURL): - # - mariadb-connector-c - # - aws-s3-cmake - # - sentry-native - set (CURL_FOUND ON CACHE BOOL "") - set (CURL_ROOT_DIR ${LIBRARY_DIR} CACHE PATH "") - set (CURL_INCLUDE_DIR ${LIBRARY_DIR}/include CACHE PATH "") - set (CURL_INCLUDE_DIRS ${LIBRARY_DIR}/include CACHE PATH "") - set (CURL_LIBRARY curl CACHE STRING "") - set (CURL_LIBRARIES ${CURL_LIBRARY} CACHE STRING "") - set (CURL_VERSION_STRING 7.67.0 CACHE STRING "") - add_library (CURL::libcurl ALIAS ${CURL_LIBRARY}) - else () - find_package (CURL REQUIRED) - endif () + target_compile_definitions (curl PRIVATE + HAVE_CONFIG_H + BUILDING_LIBCURL + CURL_HIDDEN_SYMBOLS + libcurl_EXPORTS + OS="${CMAKE_SYSTEM_NAME}" + ) + target_include_directories (curl PUBLIC + ${LIBRARY_DIR}/include + ${LIBRARY_DIR}/lib + . # curl_config.h + ) + + target_link_libraries (curl PRIVATE ssl) + + # The library is large - avoid bloat (XXX: is it?) + target_compile_options (curl PRIVATE -g0) + + # find_package(CURL) compatibility for the following packages that uses + # find_package(CURL)/include(FindCURL): + # - mariadb-connector-c + # - aws-s3-cmake + # - sentry-native + set (CURL_FOUND ON CACHE BOOL "") + set (CURL_ROOT_DIR ${LIBRARY_DIR} CACHE PATH "") + set (CURL_INCLUDE_DIR ${LIBRARY_DIR}/include CACHE PATH "") + set (CURL_INCLUDE_DIRS ${LIBRARY_DIR}/include CACHE PATH "") + set (CURL_LIBRARY curl CACHE STRING "") + set (CURL_LIBRARIES ${CURL_LIBRARY} CACHE STRING "") + set (CURL_VERSION_STRING 7.67.0 CACHE STRING "") + add_library (CURL::libcurl ALIAS ${CURL_LIBRARY}) + + set (USE_INTERNAL_CURL 1) endif () message (STATUS "Using curl: ${CURL_INCLUDE_DIRS} : ${CURL_LIBRARIES}") diff --git a/contrib/hyperscan-cmake/CMakeLists.txt b/contrib/hyperscan-cmake/CMakeLists.txt index bc2f6c4e89d..e403eeb9d3e 100644 --- a/contrib/hyperscan-cmake/CMakeLists.txt +++ b/contrib/hyperscan-cmake/CMakeLists.txt @@ -1,252 +1,272 @@ -option (ENABLE_HYPERSCAN "Enable hyperscan library" ${ENABLE_LIBRARIES}) - -if (NOT HAVE_SSSE3) +if (HAVE_SSSE3) + option (ENABLE_HYPERSCAN "Enable hyperscan library" ${ENABLE_LIBRARIES}) +elseif(ENABLE_HYPERSCAN) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use hyperscan without SSSE3") set (ENABLE_HYPERSCAN OFF) endif () -if (ENABLE_HYPERSCAN) - option (USE_INTERNAL_HYPERSCAN_LIBRARY "Use internal hyperscan library" ${NOT_UNBUNDLED}) - +if (NOT ENABLE_HYPERSCAN) if (USE_INTERNAL_HYPERSCAN_LIBRARY) - set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/hyperscan) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal hyperscan with ENABLE_HYPERSCAN=OFF") + endif() - set (SRCS - ${LIBRARY_DIR}/src/alloc.c - ${LIBRARY_DIR}/src/compiler/asserts.cpp - ${LIBRARY_DIR}/src/compiler/compiler.cpp - ${LIBRARY_DIR}/src/compiler/error.cpp - ${LIBRARY_DIR}/src/crc32.c - ${LIBRARY_DIR}/src/database.c - ${LIBRARY_DIR}/src/fdr/engine_description.cpp - ${LIBRARY_DIR}/src/fdr/fdr_compile_util.cpp - ${LIBRARY_DIR}/src/fdr/fdr_compile.cpp - ${LIBRARY_DIR}/src/fdr/fdr_confirm_compile.cpp - ${LIBRARY_DIR}/src/fdr/fdr_engine_description.cpp - ${LIBRARY_DIR}/src/fdr/fdr.c - ${LIBRARY_DIR}/src/fdr/flood_compile.cpp - ${LIBRARY_DIR}/src/fdr/teddy_compile.cpp - ${LIBRARY_DIR}/src/fdr/teddy_engine_description.cpp - ${LIBRARY_DIR}/src/fdr/teddy.c - ${LIBRARY_DIR}/src/grey.cpp - ${LIBRARY_DIR}/src/hs_valid_platform.c - ${LIBRARY_DIR}/src/hs_version.c - ${LIBRARY_DIR}/src/hs.cpp - ${LIBRARY_DIR}/src/hwlm/hwlm_build.cpp - ${LIBRARY_DIR}/src/hwlm/hwlm_literal.cpp - ${LIBRARY_DIR}/src/hwlm/hwlm.c - ${LIBRARY_DIR}/src/hwlm/noodle_build.cpp - ${LIBRARY_DIR}/src/hwlm/noodle_engine.c - ${LIBRARY_DIR}/src/nfa/accel_dfa_build_strat.cpp - ${LIBRARY_DIR}/src/nfa/accel.c - ${LIBRARY_DIR}/src/nfa/accelcompile.cpp - ${LIBRARY_DIR}/src/nfa/castle.c - ${LIBRARY_DIR}/src/nfa/castlecompile.cpp - ${LIBRARY_DIR}/src/nfa/dfa_build_strat.cpp - ${LIBRARY_DIR}/src/nfa/dfa_min.cpp - ${LIBRARY_DIR}/src/nfa/gough.c - ${LIBRARY_DIR}/src/nfa/goughcompile_accel.cpp - ${LIBRARY_DIR}/src/nfa/goughcompile_reg.cpp - ${LIBRARY_DIR}/src/nfa/goughcompile.cpp - ${LIBRARY_DIR}/src/nfa/lbr.c - ${LIBRARY_DIR}/src/nfa/limex_64.c - ${LIBRARY_DIR}/src/nfa/limex_accel.c - ${LIBRARY_DIR}/src/nfa/limex_compile.cpp - ${LIBRARY_DIR}/src/nfa/limex_native.c - ${LIBRARY_DIR}/src/nfa/limex_simd128.c - ${LIBRARY_DIR}/src/nfa/limex_simd256.c - ${LIBRARY_DIR}/src/nfa/limex_simd384.c - ${LIBRARY_DIR}/src/nfa/limex_simd512.c - ${LIBRARY_DIR}/src/nfa/mcclellan.c - ${LIBRARY_DIR}/src/nfa/mcclellancompile_util.cpp - ${LIBRARY_DIR}/src/nfa/mcclellancompile.cpp - ${LIBRARY_DIR}/src/nfa/mcsheng_compile.cpp - ${LIBRARY_DIR}/src/nfa/mcsheng_data.c - ${LIBRARY_DIR}/src/nfa/mcsheng.c - ${LIBRARY_DIR}/src/nfa/mpv.c - ${LIBRARY_DIR}/src/nfa/mpvcompile.cpp - ${LIBRARY_DIR}/src/nfa/nfa_api_dispatch.c - ${LIBRARY_DIR}/src/nfa/nfa_build_util.cpp - ${LIBRARY_DIR}/src/nfa/rdfa_graph.cpp - ${LIBRARY_DIR}/src/nfa/rdfa_merge.cpp - ${LIBRARY_DIR}/src/nfa/rdfa.cpp - ${LIBRARY_DIR}/src/nfa/repeat.c - ${LIBRARY_DIR}/src/nfa/repeatcompile.cpp - ${LIBRARY_DIR}/src/nfa/sheng.c - ${LIBRARY_DIR}/src/nfa/shengcompile.cpp - ${LIBRARY_DIR}/src/nfa/shufti.c - ${LIBRARY_DIR}/src/nfa/shufticompile.cpp - ${LIBRARY_DIR}/src/nfa/tamarama.c - ${LIBRARY_DIR}/src/nfa/tamaramacompile.cpp - ${LIBRARY_DIR}/src/nfa/truffle.c - ${LIBRARY_DIR}/src/nfa/trufflecompile.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_anchored_acyclic.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_anchored_dots.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_asserts.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_builder.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_calc_components.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_cyclic_redundancy.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_depth.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_dominators.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_edge_redundancy.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_equivalence.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_execute.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_expr_info.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_extparam.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_fixed_width.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_fuzzy.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_haig.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_holder.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_is_equal.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_lbr.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_limex_accel.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_limex.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_literal_analysis.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_literal_component.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_literal_decorated.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_mcclellan.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_misc_opt.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_netflow.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_prefilter.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_prune.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_puff.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_redundancy.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_region_redundancy.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_region.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_repeat.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_reports.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_restructuring.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_revacc.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_sep.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_small_literal_set.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_som_add_redundancy.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_som_util.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_som.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_split.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_squash.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_stop.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_uncalc_components.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_utf8.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_util.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_vacuous.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_violet.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_width.cpp - ${LIBRARY_DIR}/src/nfagraph/ng.cpp - ${LIBRARY_DIR}/src/parser/AsciiComponentClass.cpp - ${LIBRARY_DIR}/src/parser/buildstate.cpp - ${LIBRARY_DIR}/src/parser/check_refs.cpp - ${LIBRARY_DIR}/src/parser/Component.cpp - ${LIBRARY_DIR}/src/parser/ComponentAlternation.cpp - ${LIBRARY_DIR}/src/parser/ComponentAssertion.cpp - ${LIBRARY_DIR}/src/parser/ComponentAtomicGroup.cpp - ${LIBRARY_DIR}/src/parser/ComponentBackReference.cpp - ${LIBRARY_DIR}/src/parser/ComponentBoundary.cpp - ${LIBRARY_DIR}/src/parser/ComponentByte.cpp - ${LIBRARY_DIR}/src/parser/ComponentClass.cpp - ${LIBRARY_DIR}/src/parser/ComponentCondReference.cpp - ${LIBRARY_DIR}/src/parser/ComponentEmpty.cpp - ${LIBRARY_DIR}/src/parser/ComponentEUS.cpp - ${LIBRARY_DIR}/src/parser/ComponentRepeat.cpp - ${LIBRARY_DIR}/src/parser/ComponentSequence.cpp - ${LIBRARY_DIR}/src/parser/ComponentVisitor.cpp - ${LIBRARY_DIR}/src/parser/ComponentWordBoundary.cpp - ${LIBRARY_DIR}/src/parser/ConstComponentVisitor.cpp - ${LIBRARY_DIR}/src/parser/control_verbs.cpp - ${LIBRARY_DIR}/src/parser/logical_combination.cpp - ${LIBRARY_DIR}/src/parser/parse_error.cpp - ${LIBRARY_DIR}/src/parser/parser_util.cpp - ${LIBRARY_DIR}/src/parser/Parser.cpp - ${LIBRARY_DIR}/src/parser/prefilter.cpp - ${LIBRARY_DIR}/src/parser/shortcut_literal.cpp - ${LIBRARY_DIR}/src/parser/ucp_table.cpp - ${LIBRARY_DIR}/src/parser/unsupported.cpp - ${LIBRARY_DIR}/src/parser/utf8_validate.cpp - ${LIBRARY_DIR}/src/parser/Utf8ComponentClass.cpp - ${LIBRARY_DIR}/src/rose/block.c - ${LIBRARY_DIR}/src/rose/catchup.c - ${LIBRARY_DIR}/src/rose/init.c - ${LIBRARY_DIR}/src/rose/match.c - ${LIBRARY_DIR}/src/rose/program_runtime.c - ${LIBRARY_DIR}/src/rose/rose_build_add_mask.cpp - ${LIBRARY_DIR}/src/rose/rose_build_add.cpp - ${LIBRARY_DIR}/src/rose/rose_build_anchored.cpp - ${LIBRARY_DIR}/src/rose/rose_build_bytecode.cpp - ${LIBRARY_DIR}/src/rose/rose_build_castle.cpp - ${LIBRARY_DIR}/src/rose/rose_build_compile.cpp - ${LIBRARY_DIR}/src/rose/rose_build_convert.cpp - ${LIBRARY_DIR}/src/rose/rose_build_dedupe.cpp - ${LIBRARY_DIR}/src/rose/rose_build_engine_blob.cpp - ${LIBRARY_DIR}/src/rose/rose_build_exclusive.cpp - ${LIBRARY_DIR}/src/rose/rose_build_groups.cpp - ${LIBRARY_DIR}/src/rose/rose_build_infix.cpp - ${LIBRARY_DIR}/src/rose/rose_build_instructions.cpp - ${LIBRARY_DIR}/src/rose/rose_build_lit_accel.cpp - ${LIBRARY_DIR}/src/rose/rose_build_long_lit.cpp - ${LIBRARY_DIR}/src/rose/rose_build_lookaround.cpp - ${LIBRARY_DIR}/src/rose/rose_build_matchers.cpp - ${LIBRARY_DIR}/src/rose/rose_build_merge.cpp - ${LIBRARY_DIR}/src/rose/rose_build_misc.cpp - ${LIBRARY_DIR}/src/rose/rose_build_program.cpp - ${LIBRARY_DIR}/src/rose/rose_build_role_aliasing.cpp - ${LIBRARY_DIR}/src/rose/rose_build_scatter.cpp - ${LIBRARY_DIR}/src/rose/rose_build_width.cpp - ${LIBRARY_DIR}/src/rose/rose_in_util.cpp - ${LIBRARY_DIR}/src/rose/stream.c - ${LIBRARY_DIR}/src/runtime.c - ${LIBRARY_DIR}/src/scratch.c - ${LIBRARY_DIR}/src/smallwrite/smallwrite_build.cpp - ${LIBRARY_DIR}/src/som/slot_manager.cpp - ${LIBRARY_DIR}/src/som/som_runtime.c - ${LIBRARY_DIR}/src/som/som_stream.c - ${LIBRARY_DIR}/src/stream_compress.c - ${LIBRARY_DIR}/src/util/alloc.cpp - ${LIBRARY_DIR}/src/util/charreach.cpp - ${LIBRARY_DIR}/src/util/clique.cpp - ${LIBRARY_DIR}/src/util/compile_context.cpp - ${LIBRARY_DIR}/src/util/compile_error.cpp - ${LIBRARY_DIR}/src/util/cpuid_flags.c - ${LIBRARY_DIR}/src/util/depth.cpp - ${LIBRARY_DIR}/src/util/fatbit_build.cpp - ${LIBRARY_DIR}/src/util/multibit_build.cpp - ${LIBRARY_DIR}/src/util/multibit.c - ${LIBRARY_DIR}/src/util/report_manager.cpp - ${LIBRARY_DIR}/src/util/simd_utils.c - ${LIBRARY_DIR}/src/util/state_compress.c - ${LIBRARY_DIR}/src/util/target_info.cpp - ${LIBRARY_DIR}/src/util/ue2string.cpp - ) + add_library (hyperscan INTERFACE) + target_compile_definitions (hyperscan INTERFACE USE_HYPERSCAN=0) - add_library (hyperscan ${SRCS}) + message (STATUS "Not using hyperscan") + return() +endif() - target_compile_definitions (hyperscan PUBLIC USE_HYPERSCAN=1) - target_compile_options (hyperscan - PRIVATE -g0 # Library has too much debug information - -march=corei7 -O2 -fno-strict-aliasing -fno-omit-frame-pointer -fvisibility=hidden # The options from original build system - -fno-sanitize=undefined # Assume the library takes care of itself - ) - target_include_directories (hyperscan - PRIVATE - common - ${LIBRARY_DIR}/include - ) - target_include_directories (hyperscan SYSTEM PUBLIC ${LIBRARY_DIR}/src) - if (ARCH_AMD64) - target_include_directories (hyperscan PRIVATE x86_64) - endif () - target_link_libraries (hyperscan PRIVATE boost::headers_only) - else () - find_library (LIBRARY_HYPERSCAN hs) - find_path (INCLUDE_HYPERSCAN NAMES hs.h HINTS /usr/include/hs) # Ubuntu puts headers in this folder +option (USE_INTERNAL_HYPERSCAN_LIBRARY "Use internal hyperscan library" ${NOT_UNBUNDLED}) + +if (NOT USE_INTERNAL_HYPERSCAN_LIBRARY) + find_library (LIBRARY_HYPERSCAN hs) + find_path (INCLUDE_HYPERSCAN NAMES hs.h HINTS /usr/include/hs) # Ubuntu puts headers in this folder + + if (LIBRARY_HYPERSCAN AND INCLUDE_HYPERSCAN) + set (EXTERNAL_HYPERSCAN_LIBRARY_FOUND 1) add_library (hyperscan UNKNOWN IMPORTED GLOBAL) set_target_properties (hyperscan PROPERTIES IMPORTED_LOCATION ${LIBRARY_HYPERSCAN}) set_target_properties (hyperscan PROPERTIES INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_HYPERSCAN}) set_property(TARGET hyperscan APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS USE_HYPERSCAN=1) + else () + set (EXTERNAL_HYPERSCAN_LIBRARY_FOUND 0) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system hyperscan library") endif () - message (STATUS "Using hyperscan") -else () - add_library (hyperscan INTERFACE) - target_compile_definitions (hyperscan INTERFACE USE_HYPERSCAN=0) - - message (STATUS "Not using hyperscan") endif () + +if (NOT EXTERNAL_HYPERSCAN_LIBRARY_FOUND) + set (USE_INTERNAL_HYPERSCAN_LIBRARY 1) + + set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/hyperscan) + + set (SRCS + ${LIBRARY_DIR}/src/alloc.c + ${LIBRARY_DIR}/src/compiler/asserts.cpp + ${LIBRARY_DIR}/src/compiler/compiler.cpp + ${LIBRARY_DIR}/src/compiler/error.cpp + ${LIBRARY_DIR}/src/crc32.c + ${LIBRARY_DIR}/src/database.c + ${LIBRARY_DIR}/src/fdr/engine_description.cpp + ${LIBRARY_DIR}/src/fdr/fdr_compile_util.cpp + ${LIBRARY_DIR}/src/fdr/fdr_compile.cpp + ${LIBRARY_DIR}/src/fdr/fdr_confirm_compile.cpp + ${LIBRARY_DIR}/src/fdr/fdr_engine_description.cpp + ${LIBRARY_DIR}/src/fdr/fdr.c + ${LIBRARY_DIR}/src/fdr/flood_compile.cpp + ${LIBRARY_DIR}/src/fdr/teddy_compile.cpp + ${LIBRARY_DIR}/src/fdr/teddy_engine_description.cpp + ${LIBRARY_DIR}/src/fdr/teddy.c + ${LIBRARY_DIR}/src/grey.cpp + ${LIBRARY_DIR}/src/hs_valid_platform.c + ${LIBRARY_DIR}/src/hs_version.c + ${LIBRARY_DIR}/src/hs.cpp + ${LIBRARY_DIR}/src/hwlm/hwlm_build.cpp + ${LIBRARY_DIR}/src/hwlm/hwlm_literal.cpp + ${LIBRARY_DIR}/src/hwlm/hwlm.c + ${LIBRARY_DIR}/src/hwlm/noodle_build.cpp + ${LIBRARY_DIR}/src/hwlm/noodle_engine.c + ${LIBRARY_DIR}/src/nfa/accel_dfa_build_strat.cpp + ${LIBRARY_DIR}/src/nfa/accel.c + ${LIBRARY_DIR}/src/nfa/accelcompile.cpp + ${LIBRARY_DIR}/src/nfa/castle.c + ${LIBRARY_DIR}/src/nfa/castlecompile.cpp + ${LIBRARY_DIR}/src/nfa/dfa_build_strat.cpp + ${LIBRARY_DIR}/src/nfa/dfa_min.cpp + ${LIBRARY_DIR}/src/nfa/gough.c + ${LIBRARY_DIR}/src/nfa/goughcompile_accel.cpp + ${LIBRARY_DIR}/src/nfa/goughcompile_reg.cpp + ${LIBRARY_DIR}/src/nfa/goughcompile.cpp + ${LIBRARY_DIR}/src/nfa/lbr.c + ${LIBRARY_DIR}/src/nfa/limex_64.c + ${LIBRARY_DIR}/src/nfa/limex_accel.c + ${LIBRARY_DIR}/src/nfa/limex_compile.cpp + ${LIBRARY_DIR}/src/nfa/limex_native.c + ${LIBRARY_DIR}/src/nfa/limex_simd128.c + ${LIBRARY_DIR}/src/nfa/limex_simd256.c + ${LIBRARY_DIR}/src/nfa/limex_simd384.c + ${LIBRARY_DIR}/src/nfa/limex_simd512.c + ${LIBRARY_DIR}/src/nfa/mcclellan.c + ${LIBRARY_DIR}/src/nfa/mcclellancompile_util.cpp + ${LIBRARY_DIR}/src/nfa/mcclellancompile.cpp + ${LIBRARY_DIR}/src/nfa/mcsheng_compile.cpp + ${LIBRARY_DIR}/src/nfa/mcsheng_data.c + ${LIBRARY_DIR}/src/nfa/mcsheng.c + ${LIBRARY_DIR}/src/nfa/mpv.c + ${LIBRARY_DIR}/src/nfa/mpvcompile.cpp + ${LIBRARY_DIR}/src/nfa/nfa_api_dispatch.c + ${LIBRARY_DIR}/src/nfa/nfa_build_util.cpp + ${LIBRARY_DIR}/src/nfa/rdfa_graph.cpp + ${LIBRARY_DIR}/src/nfa/rdfa_merge.cpp + ${LIBRARY_DIR}/src/nfa/rdfa.cpp + ${LIBRARY_DIR}/src/nfa/repeat.c + ${LIBRARY_DIR}/src/nfa/repeatcompile.cpp + ${LIBRARY_DIR}/src/nfa/sheng.c + ${LIBRARY_DIR}/src/nfa/shengcompile.cpp + ${LIBRARY_DIR}/src/nfa/shufti.c + ${LIBRARY_DIR}/src/nfa/shufticompile.cpp + ${LIBRARY_DIR}/src/nfa/tamarama.c + ${LIBRARY_DIR}/src/nfa/tamaramacompile.cpp + ${LIBRARY_DIR}/src/nfa/truffle.c + ${LIBRARY_DIR}/src/nfa/trufflecompile.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_anchored_acyclic.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_anchored_dots.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_asserts.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_builder.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_calc_components.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_cyclic_redundancy.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_depth.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_dominators.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_edge_redundancy.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_equivalence.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_execute.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_expr_info.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_extparam.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_fixed_width.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_fuzzy.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_haig.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_holder.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_is_equal.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_lbr.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_limex_accel.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_limex.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_literal_analysis.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_literal_component.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_literal_decorated.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_mcclellan.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_misc_opt.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_netflow.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_prefilter.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_prune.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_puff.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_redundancy.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_region_redundancy.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_region.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_repeat.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_reports.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_restructuring.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_revacc.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_sep.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_small_literal_set.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_som_add_redundancy.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_som_util.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_som.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_split.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_squash.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_stop.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_uncalc_components.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_utf8.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_util.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_vacuous.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_violet.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_width.cpp + ${LIBRARY_DIR}/src/nfagraph/ng.cpp + ${LIBRARY_DIR}/src/parser/AsciiComponentClass.cpp + ${LIBRARY_DIR}/src/parser/buildstate.cpp + ${LIBRARY_DIR}/src/parser/check_refs.cpp + ${LIBRARY_DIR}/src/parser/Component.cpp + ${LIBRARY_DIR}/src/parser/ComponentAlternation.cpp + ${LIBRARY_DIR}/src/parser/ComponentAssertion.cpp + ${LIBRARY_DIR}/src/parser/ComponentAtomicGroup.cpp + ${LIBRARY_DIR}/src/parser/ComponentBackReference.cpp + ${LIBRARY_DIR}/src/parser/ComponentBoundary.cpp + ${LIBRARY_DIR}/src/parser/ComponentByte.cpp + ${LIBRARY_DIR}/src/parser/ComponentClass.cpp + ${LIBRARY_DIR}/src/parser/ComponentCondReference.cpp + ${LIBRARY_DIR}/src/parser/ComponentEmpty.cpp + ${LIBRARY_DIR}/src/parser/ComponentEUS.cpp + ${LIBRARY_DIR}/src/parser/ComponentRepeat.cpp + ${LIBRARY_DIR}/src/parser/ComponentSequence.cpp + ${LIBRARY_DIR}/src/parser/ComponentVisitor.cpp + ${LIBRARY_DIR}/src/parser/ComponentWordBoundary.cpp + ${LIBRARY_DIR}/src/parser/ConstComponentVisitor.cpp + ${LIBRARY_DIR}/src/parser/control_verbs.cpp + ${LIBRARY_DIR}/src/parser/logical_combination.cpp + ${LIBRARY_DIR}/src/parser/parse_error.cpp + ${LIBRARY_DIR}/src/parser/parser_util.cpp + ${LIBRARY_DIR}/src/parser/Parser.cpp + ${LIBRARY_DIR}/src/parser/prefilter.cpp + ${LIBRARY_DIR}/src/parser/shortcut_literal.cpp + ${LIBRARY_DIR}/src/parser/ucp_table.cpp + ${LIBRARY_DIR}/src/parser/unsupported.cpp + ${LIBRARY_DIR}/src/parser/utf8_validate.cpp + ${LIBRARY_DIR}/src/parser/Utf8ComponentClass.cpp + ${LIBRARY_DIR}/src/rose/block.c + ${LIBRARY_DIR}/src/rose/catchup.c + ${LIBRARY_DIR}/src/rose/init.c + ${LIBRARY_DIR}/src/rose/match.c + ${LIBRARY_DIR}/src/rose/program_runtime.c + ${LIBRARY_DIR}/src/rose/rose_build_add_mask.cpp + ${LIBRARY_DIR}/src/rose/rose_build_add.cpp + ${LIBRARY_DIR}/src/rose/rose_build_anchored.cpp + ${LIBRARY_DIR}/src/rose/rose_build_bytecode.cpp + ${LIBRARY_DIR}/src/rose/rose_build_castle.cpp + ${LIBRARY_DIR}/src/rose/rose_build_compile.cpp + ${LIBRARY_DIR}/src/rose/rose_build_convert.cpp + ${LIBRARY_DIR}/src/rose/rose_build_dedupe.cpp + ${LIBRARY_DIR}/src/rose/rose_build_engine_blob.cpp + ${LIBRARY_DIR}/src/rose/rose_build_exclusive.cpp + ${LIBRARY_DIR}/src/rose/rose_build_groups.cpp + ${LIBRARY_DIR}/src/rose/rose_build_infix.cpp + ${LIBRARY_DIR}/src/rose/rose_build_instructions.cpp + ${LIBRARY_DIR}/src/rose/rose_build_lit_accel.cpp + ${LIBRARY_DIR}/src/rose/rose_build_long_lit.cpp + ${LIBRARY_DIR}/src/rose/rose_build_lookaround.cpp + ${LIBRARY_DIR}/src/rose/rose_build_matchers.cpp + ${LIBRARY_DIR}/src/rose/rose_build_merge.cpp + ${LIBRARY_DIR}/src/rose/rose_build_misc.cpp + ${LIBRARY_DIR}/src/rose/rose_build_program.cpp + ${LIBRARY_DIR}/src/rose/rose_build_role_aliasing.cpp + ${LIBRARY_DIR}/src/rose/rose_build_scatter.cpp + ${LIBRARY_DIR}/src/rose/rose_build_width.cpp + ${LIBRARY_DIR}/src/rose/rose_in_util.cpp + ${LIBRARY_DIR}/src/rose/stream.c + ${LIBRARY_DIR}/src/runtime.c + ${LIBRARY_DIR}/src/scratch.c + ${LIBRARY_DIR}/src/smallwrite/smallwrite_build.cpp + ${LIBRARY_DIR}/src/som/slot_manager.cpp + ${LIBRARY_DIR}/src/som/som_runtime.c + ${LIBRARY_DIR}/src/som/som_stream.c + ${LIBRARY_DIR}/src/stream_compress.c + ${LIBRARY_DIR}/src/util/alloc.cpp + ${LIBRARY_DIR}/src/util/charreach.cpp + ${LIBRARY_DIR}/src/util/clique.cpp + ${LIBRARY_DIR}/src/util/compile_context.cpp + ${LIBRARY_DIR}/src/util/compile_error.cpp + ${LIBRARY_DIR}/src/util/cpuid_flags.c + ${LIBRARY_DIR}/src/util/depth.cpp + ${LIBRARY_DIR}/src/util/fatbit_build.cpp + ${LIBRARY_DIR}/src/util/multibit_build.cpp + ${LIBRARY_DIR}/src/util/multibit.c + ${LIBRARY_DIR}/src/util/report_manager.cpp + ${LIBRARY_DIR}/src/util/simd_utils.c + ${LIBRARY_DIR}/src/util/state_compress.c + ${LIBRARY_DIR}/src/util/target_info.cpp + ${LIBRARY_DIR}/src/util/ue2string.cpp + ) + + add_library (hyperscan ${SRCS}) + + target_compile_definitions (hyperscan PUBLIC USE_HYPERSCAN=1) + target_compile_options (hyperscan + PRIVATE -g0 # Library has too much debug information + -march=corei7 -O2 -fno-strict-aliasing -fno-omit-frame-pointer -fvisibility=hidden # The options from original build system + -fno-sanitize=undefined # Assume the library takes care of itself + ) + target_include_directories (hyperscan + PRIVATE + common + ${LIBRARY_DIR}/include + ) + target_include_directories (hyperscan SYSTEM PUBLIC ${LIBRARY_DIR}/src) + if (ARCH_AMD64) + target_include_directories (hyperscan PRIVATE x86_64) + endif () + target_link_libraries (hyperscan PRIVATE boost::headers_only) + + set (USE_INTERNAL_HYPERSCAN_LIBRARY 1) +endif () + +message (STATUS "Using hyperscan") diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 13f7ea3326b..e8a690ff149 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -2,140 +2,40 @@ option (ENABLE_JEMALLOC "Enable jemalloc allocator" ${ENABLE_LIBRARIES}) if (SANITIZE OR NOT (ARCH_AMD64 OR ARCH_ARM) OR NOT (OS_LINUX OR OS_FREEBSD OR OS_DARWIN)) set (ENABLE_JEMALLOC OFF) - message (STATUS "jemalloc is disabled implicitly: it doesn't work with sanitizers and can only be used with x86_64 or aarch64 on linux or freebsd.") + message (${RECONFIGURE_MESSAGE_LEVEL} + "jemalloc is disabled implicitly: it doesn't work with sanitizers and can only be used with x86_64 or aarch64 on linux or freebsd.") endif () -if (ENABLE_JEMALLOC) - if (NOT OS_LINUX) - message (WARNING "jemalloc support on non-linux is EXPERIMENTAL") +if (NOT ENABLE_JEMALLOC) + if(USE_INTERNAL_JEMALLOC_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal jemalloc with ENABLE_JEMALLOC=OFF") endif() - option (USE_INTERNAL_JEMALLOC "Use internal jemalloc library" ${NOT_UNBUNDLED}) + add_library(jemalloc INTERFACE) + target_compile_definitions(jemalloc INTERFACE USE_JEMALLOC=0) - if (USE_INTERNAL_JEMALLOC) - if (OS_LINUX) - # ThreadPool select job randomly, and there can be some threads that had been - # performed some memory heavy task before and will be inactive for some time, - # but until it will became active again, the memory will not be freed since by - # default each thread has it's own arena, but there should be not more then - # 4*CPU arenas (see opt.nareans description). - # - # By enabling percpu_arena number of arenas limited to number of CPUs and hence - # this problem should go away. - # - # muzzy_decay_ms -- use MADV_FREE when available on newer Linuxes, to - # avoid spurious latencies and additional work associated with - # MADV_DONTNEED. See - # https://github.com/ClickHouse/ClickHouse/issues/11121 for motivation. - set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0,muzzy_decay_ms:10000") - else() - set (JEMALLOC_CONFIG_MALLOC_CONF "oversize_threshold:0,muzzy_decay_ms:10000") - endif() - # CACHE variable is empty, to allow changing defaults without necessity - # to purge cache - set (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE "" CACHE STRING "Change default configuration string of JEMalloc" ) - if (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE) - set (JEMALLOC_CONFIG_MALLOC_CONF "${JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE}") - endif() - message (STATUS "jemalloc malloc_conf: ${JEMALLOC_CONFIG_MALLOC_CONF}") + message (STATUS "Not using jemalloc") + return() +endif () - set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/jemalloc") +if (NOT OS_LINUX) + message (WARNING "jemalloc support on non-linux is EXPERIMENTAL") +endif() - set (SRCS - ${LIBRARY_DIR}/src/arena.c - ${LIBRARY_DIR}/src/background_thread.c - ${LIBRARY_DIR}/src/base.c - ${LIBRARY_DIR}/src/bin.c - ${LIBRARY_DIR}/src/bitmap.c - ${LIBRARY_DIR}/src/ckh.c - ${LIBRARY_DIR}/src/ctl.c - ${LIBRARY_DIR}/src/div.c - ${LIBRARY_DIR}/src/extent.c - ${LIBRARY_DIR}/src/extent_dss.c - ${LIBRARY_DIR}/src/extent_mmap.c - ${LIBRARY_DIR}/src/hash.c - ${LIBRARY_DIR}/src/hook.c - ${LIBRARY_DIR}/src/jemalloc.c - ${LIBRARY_DIR}/src/large.c - ${LIBRARY_DIR}/src/log.c - ${LIBRARY_DIR}/src/malloc_io.c - ${LIBRARY_DIR}/src/mutex.c - ${LIBRARY_DIR}/src/mutex_pool.c - ${LIBRARY_DIR}/src/nstime.c - ${LIBRARY_DIR}/src/pages.c - ${LIBRARY_DIR}/src/prng.c - ${LIBRARY_DIR}/src/prof.c - ${LIBRARY_DIR}/src/rtree.c - ${LIBRARY_DIR}/src/sc.c - ${LIBRARY_DIR}/src/stats.c - ${LIBRARY_DIR}/src/sz.c - ${LIBRARY_DIR}/src/tcache.c - ${LIBRARY_DIR}/src/test_hooks.c - ${LIBRARY_DIR}/src/ticker.c - ${LIBRARY_DIR}/src/tsd.c - ${LIBRARY_DIR}/src/witness.c - ${LIBRARY_DIR}/src/safety_check.c - ) - if (OS_DARWIN) - list(APPEND SRCS ${LIBRARY_DIR}/src/zone.c) - endif () +option (USE_INTERNAL_JEMALLOC_LIBRARY "Use internal jemalloc library" ${NOT_UNBUNDLED}) - add_library(jemalloc ${SRCS}) - target_include_directories(jemalloc PRIVATE ${LIBRARY_DIR}/include) - target_include_directories(jemalloc SYSTEM PUBLIC include) +if (NOT USE_INTERNAL_JEMALLOC_LIBRARY) + find_library(LIBRARY_JEMALLOC jemalloc) + find_path(INCLUDE_JEMALLOC jemalloc/jemalloc.h) - set (JEMALLOC_INCLUDE_PREFIX) - # OS_ - if (OS_LINUX) - set (JEMALLOC_INCLUDE_PREFIX "include_linux") - elseif (OS_FREEBSD) - set (JEMALLOC_INCLUDE_PREFIX "include_freebsd") - elseif (OS_DARWIN) - set (JEMALLOC_INCLUDE_PREFIX "include_darwin") - else () - message (FATAL_ERROR "This OS is not supported") - endif () - # ARCH_ - if (ARCH_AMD64) - set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_x86_64") - elseif (ARCH_ARM) - set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_aarch64") - else () - message (FATAL_ERROR "This arch is not supported") - endif () - - configure_file(${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal/jemalloc_internal_defs.h.in - ${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal/jemalloc_internal_defs.h) - target_include_directories(jemalloc SYSTEM PRIVATE - ${CMAKE_CURRENT_BINARY_DIR}/${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal) - - target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE) - - if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") - target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_DEBUG=1 -DJEMALLOC_PROF=1) - - if (USE_UNWIND) - target_compile_definitions (jemalloc PRIVATE -DJEMALLOC_PROF_LIBUNWIND=1) - target_link_libraries (jemalloc PRIVATE unwind) - endif () - endif () - - target_compile_options(jemalloc PRIVATE -Wno-redundant-decls) - # for RTLD_NEXT - target_compile_options(jemalloc PRIVATE -D_GNU_SOURCE) - else () - find_library(LIBRARY_JEMALLOC jemalloc) - find_path(INCLUDE_JEMALLOC jemalloc/jemalloc.h) + if (LIBRARY_JEMALLOC AND INCLUDE_JEMALLOC) + set(EXTERNAL_JEMALLOC_LIBRARY_FOUND 1) set(THREADS_PREFER_PTHREAD_FLAG ON) - find_package(Threads REQUIRED) + find_package(Threads) - add_library (jemalloc STATIC IMPORTED) - set_property (TARGET jemalloc PROPERTY IMPORTED_LOCATION ${LIBRARY_JEMALLOC}) - set_property (TARGET jemalloc PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_JEMALLOC}) - set_property (TARGET jemalloc PROPERTY INTERFACE_LINK_LIBRARIES Threads::Threads dl) - - set (CMAKE_REQUIRED_LIBRARIES jemalloc) + set (CMAKE_REQUIRED_LIBRARIES ${LIBRARY_JEMALLOC} Threads::Threads "dl") + set (CMAKE_REQUIRED_INCLUDES ${INCLUDE_JEMALLOC}) check_cxx_source_compiles ( " #include @@ -144,24 +44,145 @@ if (ENABLE_JEMALLOC) free(mallocx(1, 0)); } " - EXTERNAL_JEMALLOC_WORKS + EXTERNAL_JEMALLOC_LIBRARY_WORKS ) - if (NOT EXTERNAL_JEMALLOC_WORKS) - message (FATAL_ERROR "jemalloc is unusable: ${LIBRARY_JEMALLOC} ${INCLUDE_JEMALLOC}") + if (EXTERNAL_JEMALLOC_LIBRARY_WORKS) + add_library (jemalloc STATIC IMPORTED) + set_property (TARGET jemalloc PROPERTY IMPORTED_LOCATION ${LIBRARY_JEMALLOC}) + set_property (TARGET jemalloc PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_JEMALLOC}) + set_property (TARGET jemalloc PROPERTY INTERFACE_LINK_LIBRARIES Threads::Threads dl) + else() + message (${RECONFIGURE_MESSAGE_LEVEL} "External jemalloc is unusable: ${LIBRARY_JEMALLOC} ${INCLUDE_JEMALLOC}") + endif () + + else() + set(EXTERNAL_JEMALLOC_LIBRARY_FOUND 0) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system jemalloc") + endif() +endif () + +if (NOT EXTERNAL_JEMALLOC_LIBRARY_FOUND OR NOT EXTERNAL_JEMALLOC_LIBRARY_WORKS) + set(USE_INTERNAL_JEMALLOC_LIBRARY 1) + + if (OS_LINUX) + # ThreadPool select job randomly, and there can be some threads that had been + # performed some memory heavy task before and will be inactive for some time, + # but until it will became active again, the memory will not be freed since by + # default each thread has it's own arena, but there should be not more then + # 4*CPU arenas (see opt.nareans description). + # + # By enabling percpu_arena number of arenas limited to number of CPUs and hence + # this problem should go away. + # + # muzzy_decay_ms -- use MADV_FREE when available on newer Linuxes, to + # avoid spurious latencies and additional work associated with + # MADV_DONTNEED. See + # https://github.com/ClickHouse/ClickHouse/issues/11121 for motivation. + set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0,muzzy_decay_ms:10000") + else() + set (JEMALLOC_CONFIG_MALLOC_CONF "oversize_threshold:0,muzzy_decay_ms:10000") + endif() + # CACHE variable is empty, to allow changing defaults without necessity + # to purge cache + set (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE "" CACHE STRING "Change default configuration string of JEMalloc" ) + if (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE) + set (JEMALLOC_CONFIG_MALLOC_CONF "${JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE}") + endif() + message (STATUS "jemalloc malloc_conf: ${JEMALLOC_CONFIG_MALLOC_CONF}") + + set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/jemalloc") + + set (SRCS + ${LIBRARY_DIR}/src/arena.c + ${LIBRARY_DIR}/src/background_thread.c + ${LIBRARY_DIR}/src/base.c + ${LIBRARY_DIR}/src/bin.c + ${LIBRARY_DIR}/src/bitmap.c + ${LIBRARY_DIR}/src/ckh.c + ${LIBRARY_DIR}/src/ctl.c + ${LIBRARY_DIR}/src/div.c + ${LIBRARY_DIR}/src/extent.c + ${LIBRARY_DIR}/src/extent_dss.c + ${LIBRARY_DIR}/src/extent_mmap.c + ${LIBRARY_DIR}/src/hash.c + ${LIBRARY_DIR}/src/hook.c + ${LIBRARY_DIR}/src/jemalloc.c + ${LIBRARY_DIR}/src/large.c + ${LIBRARY_DIR}/src/log.c + ${LIBRARY_DIR}/src/malloc_io.c + ${LIBRARY_DIR}/src/mutex.c + ${LIBRARY_DIR}/src/mutex_pool.c + ${LIBRARY_DIR}/src/nstime.c + ${LIBRARY_DIR}/src/pages.c + ${LIBRARY_DIR}/src/prng.c + ${LIBRARY_DIR}/src/prof.c + ${LIBRARY_DIR}/src/rtree.c + ${LIBRARY_DIR}/src/sc.c + ${LIBRARY_DIR}/src/stats.c + ${LIBRARY_DIR}/src/sz.c + ${LIBRARY_DIR}/src/tcache.c + ${LIBRARY_DIR}/src/test_hooks.c + ${LIBRARY_DIR}/src/ticker.c + ${LIBRARY_DIR}/src/tsd.c + ${LIBRARY_DIR}/src/witness.c + ${LIBRARY_DIR}/src/safety_check.c + ) + if (OS_DARWIN) + list(APPEND SRCS ${LIBRARY_DIR}/src/zone.c) + endif () + + add_library(jemalloc ${SRCS}) + target_include_directories(jemalloc PRIVATE ${LIBRARY_DIR}/include) + target_include_directories(jemalloc SYSTEM PUBLIC include) + + set (JEMALLOC_INCLUDE_PREFIX) + # OS_ + if (OS_LINUX) + set (JEMALLOC_INCLUDE_PREFIX "include_linux") + elseif (OS_FREEBSD) + set (JEMALLOC_INCLUDE_PREFIX "include_freebsd") + elseif (OS_DARWIN) + set (JEMALLOC_INCLUDE_PREFIX "include_darwin") + else () + message (FATAL_ERROR "internal jemalloc: This OS is not supported") + endif () + # ARCH_ + if (ARCH_AMD64) + set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_x86_64") + elseif (ARCH_ARM) + set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_aarch64") + else () + message (FATAL_ERROR "internal jemalloc: This arch is not supported") + endif () + + configure_file(${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal/jemalloc_internal_defs.h.in + ${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal/jemalloc_internal_defs.h) + target_include_directories(jemalloc SYSTEM PRIVATE + ${CMAKE_CURRENT_BINARY_DIR}/${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal) + + target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE) + + if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") + target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_DEBUG=1 -DJEMALLOC_PROF=1) + + if (USE_UNWIND) + target_compile_definitions (jemalloc PRIVATE -DJEMALLOC_PROF_LIBUNWIND=1) + target_link_libraries (jemalloc PRIVATE unwind) endif () endif () - set_property(TARGET jemalloc APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS USE_JEMALLOC=1) - if (MAKE_STATIC_LIBRARIES) - # To detect whether we need to register jemalloc for osx as default zone. - set_property(TARGET jemalloc APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS BUNDLED_STATIC_JEMALLOC=1) - endif() + target_compile_options(jemalloc PRIVATE -Wno-redundant-decls) + # for RTLD_NEXT + target_compile_options(jemalloc PRIVATE -D_GNU_SOURCE) - message (STATUS "Using jemalloc") -else () - add_library(jemalloc INTERFACE) - target_compile_definitions(jemalloc INTERFACE USE_JEMALLOC=0) - - message (STATUS "Not using jemalloc") + set (USE_INTERNAL_JEMALLOC_LIBRARY 1) endif () + +set_property(TARGET jemalloc APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS USE_JEMALLOC=1) +if (MAKE_STATIC_LIBRARIES) + # To detect whether we need to register jemalloc for osx as default zone. + set_property(TARGET jemalloc APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS BUNDLED_STATIC_JEMALLOC=1) +endif() + +message (STATUS "Using jemalloc") diff --git a/contrib/libcpuid-cmake/CMakeLists.txt b/contrib/libcpuid-cmake/CMakeLists.txt index cb28cbd21da..564e67994bf 100644 --- a/contrib/libcpuid-cmake/CMakeLists.txt +++ b/contrib/libcpuid-cmake/CMakeLists.txt @@ -1,35 +1,38 @@ option (ENABLE_CPUID "Enable libcpuid library (only internal)" ${ENABLE_LIBRARIES}) -if (ARCH_ARM) +if (ARCH_ARM AND ENABLE_CPUID) + message (${RECONFIGURE_MESSAGE_LEVEL} "cpuid is not supported on ARM") set (ENABLE_CPUID 0) endif () -if (ENABLE_CPUID) - set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/libcpuid) - - set (SRCS - ${LIBRARY_DIR}/libcpuid/asm-bits.c - ${LIBRARY_DIR}/libcpuid/cpuid_main.c - ${LIBRARY_DIR}/libcpuid/libcpuid_util.c - ${LIBRARY_DIR}/libcpuid/msrdriver.c - ${LIBRARY_DIR}/libcpuid/rdmsr.c - ${LIBRARY_DIR}/libcpuid/rdtsc.c - ${LIBRARY_DIR}/libcpuid/recog_amd.c - ${LIBRARY_DIR}/libcpuid/recog_intel.c - ) - - add_library (cpuid ${SRCS}) - - target_include_directories (cpuid SYSTEM PUBLIC ${LIBRARY_DIR}) - target_compile_definitions (cpuid PUBLIC USE_CPUID=1) - target_compile_definitions (cpuid PRIVATE VERSION="v0.4.1") - if (COMPILER_CLANG) - target_compile_options (cpuid PRIVATE -Wno-reserved-id-macro) - endif () - - message (STATUS "Using cpuid") -else () +if (NOT ENABLE_CPUID) add_library (cpuid INTERFACE) target_compile_definitions (cpuid INTERFACE USE_CPUID=0) + + return() +endif() + +set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/libcpuid") + +set (SRCS + ${LIBRARY_DIR}/libcpuid/asm-bits.c + ${LIBRARY_DIR}/libcpuid/cpuid_main.c + ${LIBRARY_DIR}/libcpuid/libcpuid_util.c + ${LIBRARY_DIR}/libcpuid/msrdriver.c + ${LIBRARY_DIR}/libcpuid/rdmsr.c + ${LIBRARY_DIR}/libcpuid/rdtsc.c + ${LIBRARY_DIR}/libcpuid/recog_amd.c + ${LIBRARY_DIR}/libcpuid/recog_intel.c +) + +add_library (cpuid ${SRCS}) + +target_include_directories (cpuid SYSTEM PUBLIC ${LIBRARY_DIR}) +target_compile_definitions (cpuid PUBLIC USE_CPUID=1) +target_compile_definitions (cpuid PRIVATE VERSION="v0.4.1") +if (COMPILER_CLANG) + target_compile_options (cpuid PRIVATE -Wno-reserved-id-macro) endif () + +message (STATUS "Using cpuid") diff --git a/contrib/libhdfs3-cmake/CMakeLists.txt b/contrib/libhdfs3-cmake/CMakeLists.txt index 4c71770f5b6..5b704ab1410 100644 --- a/contrib/libhdfs3-cmake/CMakeLists.txt +++ b/contrib/libhdfs3-cmake/CMakeLists.txt @@ -1,4 +1,12 @@ -if (NOT USE_INTERNAL_PROTOBUF_LIBRARY) +option(PROTOBUF_OLD_ABI_COMPAT "Set to ON for compatiability with external protobuf which was compiled old C++ ABI" ON) + +if (PROTOBUF_OLD_ABI_COMPAT) + if (NOT ENABLE_PROTOBUF OR USE_INTERNAL_PROTOBUF_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "PROTOBUF_OLD_ABI_COMPAT option is ignored") + endif() +endif() + +if (NOT USE_INTERNAL_PROTOBUF_LIBRARY AND PROTOBUF_OLD_ABI_COMPAT) # compatiable with protobuf which was compiled old C++ ABI set(CMAKE_CXX_FLAGS "-D_GLIBCXX_USE_CXX11_ABI=0") set(CMAKE_C_FLAGS "") @@ -7,7 +15,7 @@ if (NOT USE_INTERNAL_PROTOBUF_LIBRARY) endif () endif() -SET(WITH_KERBEROS false) +set(WITH_KERBEROS false) # project and source dir set(HDFS3_ROOT_DIR ${ClickHouse_SOURCE_DIR}/contrib/libhdfs3) set(HDFS3_SOURCE_DIR ${HDFS3_ROOT_DIR}/src) diff --git a/contrib/lz4-cmake/CMakeLists.txt b/contrib/lz4-cmake/CMakeLists.txt index b8121976213..374837fbe58 100644 --- a/contrib/lz4-cmake/CMakeLists.txt +++ b/contrib/lz4-cmake/CMakeLists.txt @@ -1,13 +1,30 @@ option (USE_INTERNAL_LZ4_LIBRARY "Use internal lz4 library" ${NOT_UNBUNDLED}) -if (USE_INTERNAL_LZ4_LIBRARY) - set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/lz4) +if (NOT USE_INTERNAL_LZ4_LIBRARY) + find_library (LIBRARY_LZ4 lz4) + find_path (INCLUDE_LZ4 lz4.h) + + if (LIBRARY_LZ4 AND INCLUDE_LZ4) + set(EXTERNAL_LZ4_LIBRARY_FOUND 1) + add_library (lz4 UNKNOWN IMPORTED) + set_property (TARGET lz4 PROPERTY IMPORTED_LOCATION ${LIBRARY_LZ4}) + set_property (TARGET lz4 PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_LZ4}) + set_property (TARGET lz4 APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS USE_XXHASH=0) + else() + set(EXTERNAL_LZ4_LIBRARY_FOUND 0) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system lz4") + endif() +endif() + +if (NOT EXTERNAL_LZ4_LIBRARY_FOUND) + set (USE_INTERNAL_LZ4_LIBRARY 1) + set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/lz4") set (SRCS - ${LIBRARY_DIR}/lib/lz4.c - ${LIBRARY_DIR}/lib/lz4hc.c - ${LIBRARY_DIR}/lib/lz4frame.c - ${LIBRARY_DIR}/lib/xxhash.c + "${LIBRARY_DIR}/lib/lz4.c" + "${LIBRARY_DIR}/lib/lz4hc.c" + "${LIBRARY_DIR}/lib/lz4frame.c" + "${LIBRARY_DIR}/lib/xxhash.c" ) add_library (lz4 ${SRCS}) @@ -17,12 +34,4 @@ if (USE_INTERNAL_LZ4_LIBRARY) target_compile_options (lz4 PRIVATE -fno-sanitize=undefined) endif () target_include_directories(lz4 PUBLIC ${LIBRARY_DIR}/lib) -else () - find_library (LIBRARY_LZ4 lz4) - find_path (INCLUDE_LZ4 lz4.h) - - add_library (lz4 UNKNOWN IMPORTED) - set_property (TARGET lz4 PROPERTY IMPORTED_LOCATION ${LIBRARY_LZ4}) - set_property (TARGET lz4 PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_LZ4}) - set_property (TARGET lz4 APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS USE_XXHASH=0) endif () diff --git a/contrib/poco-cmake/CMakeLists.txt b/contrib/poco-cmake/CMakeLists.txt index 59b6c84a1d1..a96b07085ba 100644 --- a/contrib/poco-cmake/CMakeLists.txt +++ b/contrib/poco-cmake/CMakeLists.txt @@ -4,6 +4,9 @@ if (USE_INTERNAL_POCO_LIBRARY) set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/poco) else () find_path (ROOT_DIR NAMES Foundation/include/Poco/Poco.h include/Poco/Poco.h) + if (NOT ROOT_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system poco") + endif() endif () add_subdirectory (Crypto) diff --git a/contrib/replxx-cmake/CMakeLists.txt b/contrib/replxx-cmake/CMakeLists.txt index 48d7e8bb36b..ed6bdaa25bc 100644 --- a/contrib/replxx-cmake/CMakeLists.txt +++ b/contrib/replxx-cmake/CMakeLists.txt @@ -1,35 +1,26 @@ option (ENABLE_REPLXX "Enable replxx support" ${ENABLE_LIBRARIES}) -if (ENABLE_REPLXX) - option (USE_INTERNAL_REPLXX "Use internal replxx library" ${NOT_UNBUNDLED}) +if (NOT ENABLE_REPLXX) + if (USE_INTERNAL_REPLXX_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal replxx with ENABLE_REPLXX=OFF") + endif() - if (USE_INTERNAL_REPLXX) - set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/replxx") + add_library(replxx INTERFACE) + target_compile_definitions(replxx INTERFACE USE_REPLXX=0) - set(SRCS - ${LIBRARY_DIR}/src/conversion.cxx - ${LIBRARY_DIR}/src/ConvertUTF.cpp - ${LIBRARY_DIR}/src/escape.cxx - ${LIBRARY_DIR}/src/history.cxx - ${LIBRARY_DIR}/src/io.cxx - ${LIBRARY_DIR}/src/prompt.cxx - ${LIBRARY_DIR}/src/replxx_impl.cxx - ${LIBRARY_DIR}/src/replxx.cxx - ${LIBRARY_DIR}/src/util.cxx - ${LIBRARY_DIR}/src/wcwidth.cpp - ) + message (STATUS "Not using replxx (Beware! Runtime fallback to readline is possible!)") + return() +endif() - add_library (replxx ${SRCS}) - target_include_directories(replxx SYSTEM PUBLIC ${LIBRARY_DIR}/include) - else () - find_library(LIBRARY_REPLXX NAMES replxx replxx-static) - find_path(INCLUDE_REPLXX replxx.hxx) +option (USE_INTERNAL_REPLXX_LIBRARY "Use internal replxx library" ${NOT_UNBUNDLED}) - add_library(replxx UNKNOWN IMPORTED) - set_property(TARGET replxx PROPERTY IMPORTED_LOCATION ${LIBRARY_REPLXX}) - target_include_directories(replxx SYSTEM PUBLIC ${INCLUDE_REPLXX}) +if (NOT USE_INTERNAL_REPLXX_LIBRARY) + find_library(LIBRARY_REPLXX NAMES replxx replxx-static) + find_path(INCLUDE_REPLXX replxx.hxx) - set(CMAKE_REQUIRED_LIBRARIES replxx) + if (LIBRARY_REPLXX AND INCLUDE_REPLXX) + set(CMAKE_REQUIRED_LIBRARIES ${LIBRARY_REPLXX}) + set(CMAKE_REQUIRED_INCLUDES ${INCLUDE_REPLXX}) check_cxx_source_compiles( " #include @@ -41,20 +32,43 @@ if (ENABLE_REPLXX) ) if (NOT EXTERNAL_REPLXX_WORKS) - message (FATAL_ERROR "replxx is unusable: ${LIBRARY_REPLXX} ${INCLUDE_REPLXX}") - endif () - endif () + message (${RECONFIGURE_MESSAGE_LEVEL} "replxx is unusable: ${LIBRARY_REPLXX} ${INCLUDE_REPLXX}") + else() + add_library(replxx UNKNOWN IMPORTED) + set_property(TARGET replxx PROPERTY IMPORTED_LOCATION ${LIBRARY_REPLXX}) + target_include_directories(replxx SYSTEM PUBLIC ${INCLUDE_REPLXX}) + endif() + else() + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system replxx") + endif() +endif() - if (COMPILER_CLANG) - target_compile_options(replxx PRIVATE -Wno-documentation) - endif () - target_compile_definitions(replxx PUBLIC USE_REPLXX=1) +if (NOT LIBRARY_REPLXX OR NOT INCLUDE_REPLXX OR NOT EXTERNAL_REPLXX_WORKS) + set(USE_INTERNAL_REPLXX_LIBRARY 1) + set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/replxx") - message (STATUS "Using replxx") -else () - add_library(replxx INTERFACE) - target_compile_definitions(replxx INTERFACE USE_REPLXX=0) + set(SRCS + ${LIBRARY_DIR}/src/conversion.cxx + ${LIBRARY_DIR}/src/ConvertUTF.cpp + ${LIBRARY_DIR}/src/escape.cxx + ${LIBRARY_DIR}/src/history.cxx + ${LIBRARY_DIR}/src/io.cxx + ${LIBRARY_DIR}/src/prompt.cxx + ${LIBRARY_DIR}/src/replxx_impl.cxx + ${LIBRARY_DIR}/src/replxx.cxx + ${LIBRARY_DIR}/src/util.cxx + ${LIBRARY_DIR}/src/wcwidth.cpp + ) - message (STATUS "Not using replxx (Beware! Runtime fallback to readline is possible!)") + add_library (replxx ${SRCS}) + target_include_directories(replxx SYSTEM PUBLIC ${LIBRARY_DIR}/include) endif () + +if (COMPILER_CLANG) + target_compile_options(replxx PRIVATE -Wno-documentation) +endif () + +target_compile_definitions(replxx PUBLIC USE_REPLXX=1) + +message (STATUS "Using replxx") diff --git a/contrib/unixodbc-cmake/CMakeLists.txt b/contrib/unixodbc-cmake/CMakeLists.txt index 658fa3329d3..7a585161cc1 100644 --- a/contrib/unixodbc-cmake/CMakeLists.txt +++ b/contrib/unixodbc-cmake/CMakeLists.txt @@ -1,318 +1,336 @@ option (ENABLE_ODBC "Enable ODBC library" ${ENABLE_LIBRARIES}) if (NOT OS_LINUX) + if (ENABLE_ODBC) + message(STATUS "ODBC is only supported on Linux") + endif() set (ENABLE_ODBC OFF CACHE INTERNAL "") endif () -if (ENABLE_ODBC) - option (USE_INTERNAL_ODBC_LIBRARY "Use internal ODBC library" ${NOT_UNBUNDLED}) - +if (NOT ENABLE_ODBC) if (USE_INTERNAL_ODBC_LIBRARY) - set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/unixodbc) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal ODBC with ENABLE_ODBC=OFF") + endif() - # ltdl - - set (SRCS_LTDL - # This file is generated by 'libtool' inside libltdl directory and then removed. - linux_x86_64/libltdl/libltdlcS.c - - ${LIBRARY_DIR}/libltdl/lt__alloc.c - ${LIBRARY_DIR}/libltdl/lt__strl.c - ${LIBRARY_DIR}/libltdl/ltdl.c - ${LIBRARY_DIR}/libltdl/lt_dlloader.c - ${LIBRARY_DIR}/libltdl/slist.c - ${LIBRARY_DIR}/libltdl/lt_error.c - ${LIBRARY_DIR}/libltdl/loaders/dlopen.c - ${LIBRARY_DIR}/libltdl/loaders/preopen.c - ) - - add_library (ltdl ${SRCS_LTDL}) - - target_include_directories(ltdl - PRIVATE - linux_x86_64/libltdl - PUBLIC - ${LIBRARY_DIR}/libltdl - ${LIBRARY_DIR}/libltdl/libltdl - ) - target_compile_definitions(ltdl PRIVATE -DHAVE_CONFIG_H -DLTDL -DLTDLOPEN=libltdlc) - target_compile_options(ltdl PRIVATE -Wno-constant-logical-operand -Wno-unknown-warning-option -O2) - - # odbc - - set (SRCS - ${LIBRARY_DIR}/DriverManager/__attribute.c - ${LIBRARY_DIR}/DriverManager/__connection.c - ${LIBRARY_DIR}/DriverManager/__handles.c - ${LIBRARY_DIR}/DriverManager/__info.c - ${LIBRARY_DIR}/DriverManager/__stats.c - ${LIBRARY_DIR}/DriverManager/SQLAllocConnect.c - ${LIBRARY_DIR}/DriverManager/SQLAllocEnv.c - ${LIBRARY_DIR}/DriverManager/SQLAllocHandle.c - ${LIBRARY_DIR}/DriverManager/SQLAllocHandleStd.c - ${LIBRARY_DIR}/DriverManager/SQLAllocStmt.c - ${LIBRARY_DIR}/DriverManager/SQLBindCol.c - ${LIBRARY_DIR}/DriverManager/SQLBindParam.c - ${LIBRARY_DIR}/DriverManager/SQLBindParameter.c - ${LIBRARY_DIR}/DriverManager/SQLBrowseConnect.c - ${LIBRARY_DIR}/DriverManager/SQLBrowseConnectW.c - ${LIBRARY_DIR}/DriverManager/SQLBulkOperations.c - ${LIBRARY_DIR}/DriverManager/SQLCancel.c - ${LIBRARY_DIR}/DriverManager/SQLCancelHandle.c - ${LIBRARY_DIR}/DriverManager/SQLCloseCursor.c - ${LIBRARY_DIR}/DriverManager/SQLColAttribute.c - ${LIBRARY_DIR}/DriverManager/SQLColAttributes.c - ${LIBRARY_DIR}/DriverManager/SQLColAttributesW.c - ${LIBRARY_DIR}/DriverManager/SQLColAttributeW.c - ${LIBRARY_DIR}/DriverManager/SQLColumnPrivileges.c - ${LIBRARY_DIR}/DriverManager/SQLColumnPrivilegesW.c - ${LIBRARY_DIR}/DriverManager/SQLColumns.c - ${LIBRARY_DIR}/DriverManager/SQLColumnsW.c - ${LIBRARY_DIR}/DriverManager/SQLConnect.c - ${LIBRARY_DIR}/DriverManager/SQLConnectW.c - ${LIBRARY_DIR}/DriverManager/SQLCopyDesc.c - ${LIBRARY_DIR}/DriverManager/SQLDataSources.c - ${LIBRARY_DIR}/DriverManager/SQLDataSourcesW.c - ${LIBRARY_DIR}/DriverManager/SQLDescribeCol.c - ${LIBRARY_DIR}/DriverManager/SQLDescribeColW.c - ${LIBRARY_DIR}/DriverManager/SQLDescribeParam.c - ${LIBRARY_DIR}/DriverManager/SQLDisconnect.c - ${LIBRARY_DIR}/DriverManager/SQLDriverConnect.c - ${LIBRARY_DIR}/DriverManager/SQLDriverConnectW.c - ${LIBRARY_DIR}/DriverManager/SQLDrivers.c - ${LIBRARY_DIR}/DriverManager/SQLDriversW.c - ${LIBRARY_DIR}/DriverManager/SQLEndTran.c - ${LIBRARY_DIR}/DriverManager/SQLError.c - ${LIBRARY_DIR}/DriverManager/SQLErrorW.c - ${LIBRARY_DIR}/DriverManager/SQLExecDirect.c - ${LIBRARY_DIR}/DriverManager/SQLExecDirectW.c - ${LIBRARY_DIR}/DriverManager/SQLExecute.c - ${LIBRARY_DIR}/DriverManager/SQLExtendedFetch.c - ${LIBRARY_DIR}/DriverManager/SQLFetch.c - ${LIBRARY_DIR}/DriverManager/SQLFetchScroll.c - ${LIBRARY_DIR}/DriverManager/SQLForeignKeys.c - ${LIBRARY_DIR}/DriverManager/SQLForeignKeysW.c - ${LIBRARY_DIR}/DriverManager/SQLFreeConnect.c - ${LIBRARY_DIR}/DriverManager/SQLFreeEnv.c - ${LIBRARY_DIR}/DriverManager/SQLFreeHandle.c - ${LIBRARY_DIR}/DriverManager/SQLFreeStmt.c - ${LIBRARY_DIR}/DriverManager/SQLGetConnectAttr.c - ${LIBRARY_DIR}/DriverManager/SQLGetConnectAttrW.c - ${LIBRARY_DIR}/DriverManager/SQLGetConnectOption.c - ${LIBRARY_DIR}/DriverManager/SQLGetConnectOptionW.c - ${LIBRARY_DIR}/DriverManager/SQLGetCursorName.c - ${LIBRARY_DIR}/DriverManager/SQLGetCursorNameW.c - ${LIBRARY_DIR}/DriverManager/SQLGetData.c - ${LIBRARY_DIR}/DriverManager/SQLGetDescField.c - ${LIBRARY_DIR}/DriverManager/SQLGetDescFieldW.c - ${LIBRARY_DIR}/DriverManager/SQLGetDescRec.c - ${LIBRARY_DIR}/DriverManager/SQLGetDescRecW.c - ${LIBRARY_DIR}/DriverManager/SQLGetDiagField.c - ${LIBRARY_DIR}/DriverManager/SQLGetDiagFieldW.c - ${LIBRARY_DIR}/DriverManager/SQLGetDiagRec.c - ${LIBRARY_DIR}/DriverManager/SQLGetDiagRecW.c - ${LIBRARY_DIR}/DriverManager/SQLGetEnvAttr.c - ${LIBRARY_DIR}/DriverManager/SQLGetFunctions.c - ${LIBRARY_DIR}/DriverManager/SQLGetInfo.c - ${LIBRARY_DIR}/DriverManager/SQLGetInfoW.c - ${LIBRARY_DIR}/DriverManager/SQLGetStmtAttr.c - ${LIBRARY_DIR}/DriverManager/SQLGetStmtAttrW.c - ${LIBRARY_DIR}/DriverManager/SQLGetStmtOption.c - ${LIBRARY_DIR}/DriverManager/SQLGetTypeInfo.c - ${LIBRARY_DIR}/DriverManager/SQLGetTypeInfoW.c - ${LIBRARY_DIR}/DriverManager/SQLMoreResults.c - ${LIBRARY_DIR}/DriverManager/SQLNativeSql.c - ${LIBRARY_DIR}/DriverManager/SQLNativeSqlW.c - ${LIBRARY_DIR}/DriverManager/SQLNumParams.c - ${LIBRARY_DIR}/DriverManager/SQLNumResultCols.c - ${LIBRARY_DIR}/DriverManager/SQLParamData.c - ${LIBRARY_DIR}/DriverManager/SQLParamOptions.c - ${LIBRARY_DIR}/DriverManager/SQLPrepare.c - ${LIBRARY_DIR}/DriverManager/SQLPrepareW.c - ${LIBRARY_DIR}/DriverManager/SQLPrimaryKeys.c - ${LIBRARY_DIR}/DriverManager/SQLPrimaryKeysW.c - ${LIBRARY_DIR}/DriverManager/SQLProcedureColumns.c - ${LIBRARY_DIR}/DriverManager/SQLProcedureColumnsW.c - ${LIBRARY_DIR}/DriverManager/SQLProcedures.c - ${LIBRARY_DIR}/DriverManager/SQLProceduresW.c - ${LIBRARY_DIR}/DriverManager/SQLPutData.c - ${LIBRARY_DIR}/DriverManager/SQLRowCount.c - ${LIBRARY_DIR}/DriverManager/SQLSetConnectAttr.c - ${LIBRARY_DIR}/DriverManager/SQLSetConnectAttrW.c - ${LIBRARY_DIR}/DriverManager/SQLSetConnectOption.c - ${LIBRARY_DIR}/DriverManager/SQLSetConnectOptionW.c - ${LIBRARY_DIR}/DriverManager/SQLSetCursorName.c - ${LIBRARY_DIR}/DriverManager/SQLSetCursorNameW.c - ${LIBRARY_DIR}/DriverManager/SQLSetDescField.c - ${LIBRARY_DIR}/DriverManager/SQLSetDescFieldW.c - ${LIBRARY_DIR}/DriverManager/SQLSetDescRec.c - ${LIBRARY_DIR}/DriverManager/SQLSetEnvAttr.c - ${LIBRARY_DIR}/DriverManager/SQLSetParam.c - ${LIBRARY_DIR}/DriverManager/SQLSetPos.c - ${LIBRARY_DIR}/DriverManager/SQLSetScrollOptions.c - ${LIBRARY_DIR}/DriverManager/SQLSetStmtAttr.c - ${LIBRARY_DIR}/DriverManager/SQLSetStmtAttrW.c - ${LIBRARY_DIR}/DriverManager/SQLSetStmtOption.c - ${LIBRARY_DIR}/DriverManager/SQLSetStmtOptionW.c - ${LIBRARY_DIR}/DriverManager/SQLSpecialColumns.c - ${LIBRARY_DIR}/DriverManager/SQLSpecialColumnsW.c - ${LIBRARY_DIR}/DriverManager/SQLStatistics.c - ${LIBRARY_DIR}/DriverManager/SQLStatisticsW.c - ${LIBRARY_DIR}/DriverManager/SQLTablePrivileges.c - ${LIBRARY_DIR}/DriverManager/SQLTablePrivilegesW.c - ${LIBRARY_DIR}/DriverManager/SQLTables.c - ${LIBRARY_DIR}/DriverManager/SQLTablesW.c - ${LIBRARY_DIR}/DriverManager/SQLTransact.c - ${LIBRARY_DIR}/ini/_iniDump.c - ${LIBRARY_DIR}/ini/_iniObjectRead.c - ${LIBRARY_DIR}/ini/_iniPropertyRead.c - ${LIBRARY_DIR}/ini/_iniScanUntilObject.c - ${LIBRARY_DIR}/ini/iniAllTrim.c - ${LIBRARY_DIR}/ini/iniAppend.c - ${LIBRARY_DIR}/ini/iniClose.c - ${LIBRARY_DIR}/ini/iniCommit.c - ${LIBRARY_DIR}/ini/iniCursor.c - ${LIBRARY_DIR}/ini/iniDelete.c - ${LIBRARY_DIR}/ini/iniElement.c - ${LIBRARY_DIR}/ini/iniElementCount.c - ${LIBRARY_DIR}/ini/iniGetBookmark.c - ${LIBRARY_DIR}/ini/iniGotoBookmark.c - ${LIBRARY_DIR}/ini/iniObject.c - ${LIBRARY_DIR}/ini/iniObjectDelete.c - ${LIBRARY_DIR}/ini/iniObjectEOL.c - ${LIBRARY_DIR}/ini/iniObjectFirst.c - ${LIBRARY_DIR}/ini/iniObjectInsert.c - ${LIBRARY_DIR}/ini/iniObjectLast.c - ${LIBRARY_DIR}/ini/iniObjectNext.c - ${LIBRARY_DIR}/ini/iniObjectSeek.c - ${LIBRARY_DIR}/ini/iniObjectSeekSure.c - ${LIBRARY_DIR}/ini/iniObjectUpdate.c - ${LIBRARY_DIR}/ini/iniOpen.c - ${LIBRARY_DIR}/ini/iniProperty.c - ${LIBRARY_DIR}/ini/iniPropertyDelete.c - ${LIBRARY_DIR}/ini/iniPropertyEOL.c - ${LIBRARY_DIR}/ini/iniPropertyFirst.c - ${LIBRARY_DIR}/ini/iniPropertyInsert.c - ${LIBRARY_DIR}/ini/iniPropertyLast.c - ${LIBRARY_DIR}/ini/iniPropertyNext.c - ${LIBRARY_DIR}/ini/iniPropertySeek.c - ${LIBRARY_DIR}/ini/iniPropertySeekSure.c - ${LIBRARY_DIR}/ini/iniPropertyUpdate.c - ${LIBRARY_DIR}/ini/iniPropertyValue.c - ${LIBRARY_DIR}/ini/iniToUpper.c - ${LIBRARY_DIR}/ini/iniValue.c - ${LIBRARY_DIR}/log/_logFreeMsg.c - ${LIBRARY_DIR}/log/logClear.c - ${LIBRARY_DIR}/log/logClose.c - ${LIBRARY_DIR}/log/logOn.c - ${LIBRARY_DIR}/log/logOpen.c - ${LIBRARY_DIR}/log/logPeekMsg.c - ${LIBRARY_DIR}/log/logPopMsg.c - ${LIBRARY_DIR}/log/logPushMsg.c - ${LIBRARY_DIR}/lst/_lstAdjustCurrent.c - ${LIBRARY_DIR}/lst/_lstDump.c - ${LIBRARY_DIR}/lst/_lstFreeItem.c - ${LIBRARY_DIR}/lst/_lstNextValidItem.c - ${LIBRARY_DIR}/lst/_lstPrevValidItem.c - ${LIBRARY_DIR}/lst/_lstVisible.c - ${LIBRARY_DIR}/lst/lstAppend.c - ${LIBRARY_DIR}/lst/lstClose.c - ${LIBRARY_DIR}/lst/lstDelete.c - ${LIBRARY_DIR}/lst/lstEOL.c - ${LIBRARY_DIR}/lst/lstFirst.c - ${LIBRARY_DIR}/lst/lstGet.c - ${LIBRARY_DIR}/lst/lstGetBookMark.c - ${LIBRARY_DIR}/lst/lstGoto.c - ${LIBRARY_DIR}/lst/lstGotoBookMark.c - ${LIBRARY_DIR}/lst/lstInsert.c - ${LIBRARY_DIR}/lst/lstLast.c - ${LIBRARY_DIR}/lst/lstNext.c - ${LIBRARY_DIR}/lst/lstOpen.c - ${LIBRARY_DIR}/lst/lstOpenCursor.c - ${LIBRARY_DIR}/lst/lstPrev.c - ${LIBRARY_DIR}/lst/lstSeek.c - ${LIBRARY_DIR}/lst/lstSeekItem.c - ${LIBRARY_DIR}/lst/lstSet.c - ${LIBRARY_DIR}/lst/lstSetFreeFunc.c - ${LIBRARY_DIR}/odbcinst/_logging.c - ${LIBRARY_DIR}/odbcinst/_odbcinst_ConfigModeINI.c - ${LIBRARY_DIR}/odbcinst/_odbcinst_GetEntries.c - ${LIBRARY_DIR}/odbcinst/_odbcinst_GetSections.c - ${LIBRARY_DIR}/odbcinst/_odbcinst_SystemINI.c - ${LIBRARY_DIR}/odbcinst/_odbcinst_UserINI.c - ${LIBRARY_DIR}/odbcinst/_SQLDriverConnectPrompt.c - ${LIBRARY_DIR}/odbcinst/_SQLGetInstalledDrivers.c - ${LIBRARY_DIR}/odbcinst/_SQLWriteInstalledDrivers.c - ${LIBRARY_DIR}/odbcinst/ODBCINSTConstructProperties.c - ${LIBRARY_DIR}/odbcinst/ODBCINSTDestructProperties.c - ${LIBRARY_DIR}/odbcinst/ODBCINSTSetProperty.c - ${LIBRARY_DIR}/odbcinst/ODBCINSTValidateProperties.c - ${LIBRARY_DIR}/odbcinst/ODBCINSTValidateProperty.c - ${LIBRARY_DIR}/odbcinst/SQLConfigDataSource.c - ${LIBRARY_DIR}/odbcinst/SQLConfigDriver.c - ${LIBRARY_DIR}/odbcinst/SQLCreateDataSource.c - ${LIBRARY_DIR}/odbcinst/SQLGetAvailableDrivers.c - ${LIBRARY_DIR}/odbcinst/SQLGetConfigMode.c - ${LIBRARY_DIR}/odbcinst/SQLGetInstalledDrivers.c - ${LIBRARY_DIR}/odbcinst/SQLGetPrivateProfileString.c - ${LIBRARY_DIR}/odbcinst/SQLGetTranslator.c - ${LIBRARY_DIR}/odbcinst/SQLInstallDriverEx.c - ${LIBRARY_DIR}/odbcinst/SQLInstallDriverManager.c - ${LIBRARY_DIR}/odbcinst/SQLInstallerError.c - ${LIBRARY_DIR}/odbcinst/SQLInstallODBC.c - ${LIBRARY_DIR}/odbcinst/SQLInstallTranslatorEx.c - ${LIBRARY_DIR}/odbcinst/SQLManageDataSources.c - ${LIBRARY_DIR}/odbcinst/SQLPostInstallerError.c - ${LIBRARY_DIR}/odbcinst/SQLReadFileDSN.c - ${LIBRARY_DIR}/odbcinst/SQLRemoveDriver.c - ${LIBRARY_DIR}/odbcinst/SQLRemoveDriverManager.c - ${LIBRARY_DIR}/odbcinst/SQLRemoveDSNFromIni.c - ${LIBRARY_DIR}/odbcinst/SQLRemoveTranslator.c - ${LIBRARY_DIR}/odbcinst/SQLSetConfigMode.c - ${LIBRARY_DIR}/odbcinst/SQLValidDSN.c - ${LIBRARY_DIR}/odbcinst/SQLWriteDSNToIni.c - ${LIBRARY_DIR}/odbcinst/SQLWriteFileDSN.c - ${LIBRARY_DIR}/odbcinst/SQLWritePrivateProfileString.c - ) - - add_library (unixodbc ${SRCS}) - - target_link_libraries (unixodbc PRIVATE ltdl) - - # SYSTEM_FILE_PATH was changed to /etc - - target_include_directories (unixodbc - PRIVATE - linux_x86_64/private - PUBLIC - linux_x86_64 - ${LIBRARY_DIR}/include - ) - target_compile_definitions (unixodbc PRIVATE -DHAVE_CONFIG_H) - target_compile_options (unixodbc - PRIVATE - -Wno-dangling-else - -Wno-parentheses - -Wno-misleading-indentation - -Wno-unknown-warning-option - -Wno-reserved-id-macro - -O2 - ) - else () - add_library (unixodbc UNKNOWN IMPORTED) - - find_library (LIBRARY_ODBC unixodbc) - find_path (INCLUDE_ODBC sql.h) - set_target_properties (unixodbc PROPERTIES IMPORTED_LOCATION ${LIBRARY_ODBC}) - set_target_properties (unixodbc PROPERTIES INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_ODBC}) - endif () - - target_compile_definitions (unixodbc INTERFACE USE_ODBC=1) - - message (STATUS "Using unixodbc") -else () add_library (unixodbc INTERFACE) target_compile_definitions (unixodbc INTERFACE USE_ODBC=0) message (STATUS "Not using unixodbc") + return() +endif() + +option (USE_INTERNAL_ODBC_LIBRARY "Use internal ODBC library" ${NOT_UNBUNDLED}) + +if (NOT USE_INTERNAL_ODBC_LIBRARY) + find_library (LIBRARY_ODBC unixodbc) + find_path (INCLUDE_ODBC sql.h) + + if(LIBRARY_ODBC AND INCLUDE_ODBC) + add_library (unixodbc UNKNOWN IMPORTED) + set_target_properties (unixodbc PROPERTIES IMPORTED_LOCATION ${LIBRARY_ODBC}) + set_target_properties (unixodbc PROPERTIES INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_ODBC}) + set(EXTERNAL_ODBC_LIBRARY_FOUND 1) + else() + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system ODBC library") + set(EXTERNAL_ODBC_LIBRARY_FOUND 0) + endif() +endif() + +if (NOT EXTERNAL_ODBC_LIBRARY_FOUND) + set (USE_INTERNAL_ODBC_LIBRARY 1) + + set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/unixodbc) + + # ltdl + + set (SRCS_LTDL + # This file is generated by 'libtool' inside libltdl directory and then removed. + linux_x86_64/libltdl/libltdlcS.c + + ${LIBRARY_DIR}/libltdl/lt__alloc.c + ${LIBRARY_DIR}/libltdl/lt__strl.c + ${LIBRARY_DIR}/libltdl/ltdl.c + ${LIBRARY_DIR}/libltdl/lt_dlloader.c + ${LIBRARY_DIR}/libltdl/slist.c + ${LIBRARY_DIR}/libltdl/lt_error.c + ${LIBRARY_DIR}/libltdl/loaders/dlopen.c + ${LIBRARY_DIR}/libltdl/loaders/preopen.c + ) + + add_library (ltdl ${SRCS_LTDL}) + + target_include_directories(ltdl + PRIVATE + linux_x86_64/libltdl + PUBLIC + ${LIBRARY_DIR}/libltdl + ${LIBRARY_DIR}/libltdl/libltdl + ) + target_compile_definitions(ltdl PRIVATE -DHAVE_CONFIG_H -DLTDL -DLTDLOPEN=libltdlc) + target_compile_options(ltdl PRIVATE -Wno-constant-logical-operand -Wno-unknown-warning-option -O2) + + # odbc + + set (SRCS + ${LIBRARY_DIR}/DriverManager/__attribute.c + ${LIBRARY_DIR}/DriverManager/__connection.c + ${LIBRARY_DIR}/DriverManager/__handles.c + ${LIBRARY_DIR}/DriverManager/__info.c + ${LIBRARY_DIR}/DriverManager/__stats.c + ${LIBRARY_DIR}/DriverManager/SQLAllocConnect.c + ${LIBRARY_DIR}/DriverManager/SQLAllocEnv.c + ${LIBRARY_DIR}/DriverManager/SQLAllocHandle.c + ${LIBRARY_DIR}/DriverManager/SQLAllocHandleStd.c + ${LIBRARY_DIR}/DriverManager/SQLAllocStmt.c + ${LIBRARY_DIR}/DriverManager/SQLBindCol.c + ${LIBRARY_DIR}/DriverManager/SQLBindParam.c + ${LIBRARY_DIR}/DriverManager/SQLBindParameter.c + ${LIBRARY_DIR}/DriverManager/SQLBrowseConnect.c + ${LIBRARY_DIR}/DriverManager/SQLBrowseConnectW.c + ${LIBRARY_DIR}/DriverManager/SQLBulkOperations.c + ${LIBRARY_DIR}/DriverManager/SQLCancel.c + ${LIBRARY_DIR}/DriverManager/SQLCancelHandle.c + ${LIBRARY_DIR}/DriverManager/SQLCloseCursor.c + ${LIBRARY_DIR}/DriverManager/SQLColAttribute.c + ${LIBRARY_DIR}/DriverManager/SQLColAttributes.c + ${LIBRARY_DIR}/DriverManager/SQLColAttributesW.c + ${LIBRARY_DIR}/DriverManager/SQLColAttributeW.c + ${LIBRARY_DIR}/DriverManager/SQLColumnPrivileges.c + ${LIBRARY_DIR}/DriverManager/SQLColumnPrivilegesW.c + ${LIBRARY_DIR}/DriverManager/SQLColumns.c + ${LIBRARY_DIR}/DriverManager/SQLColumnsW.c + ${LIBRARY_DIR}/DriverManager/SQLConnect.c + ${LIBRARY_DIR}/DriverManager/SQLConnectW.c + ${LIBRARY_DIR}/DriverManager/SQLCopyDesc.c + ${LIBRARY_DIR}/DriverManager/SQLDataSources.c + ${LIBRARY_DIR}/DriverManager/SQLDataSourcesW.c + ${LIBRARY_DIR}/DriverManager/SQLDescribeCol.c + ${LIBRARY_DIR}/DriverManager/SQLDescribeColW.c + ${LIBRARY_DIR}/DriverManager/SQLDescribeParam.c + ${LIBRARY_DIR}/DriverManager/SQLDisconnect.c + ${LIBRARY_DIR}/DriverManager/SQLDriverConnect.c + ${LIBRARY_DIR}/DriverManager/SQLDriverConnectW.c + ${LIBRARY_DIR}/DriverManager/SQLDrivers.c + ${LIBRARY_DIR}/DriverManager/SQLDriversW.c + ${LIBRARY_DIR}/DriverManager/SQLEndTran.c + ${LIBRARY_DIR}/DriverManager/SQLError.c + ${LIBRARY_DIR}/DriverManager/SQLErrorW.c + ${LIBRARY_DIR}/DriverManager/SQLExecDirect.c + ${LIBRARY_DIR}/DriverManager/SQLExecDirectW.c + ${LIBRARY_DIR}/DriverManager/SQLExecute.c + ${LIBRARY_DIR}/DriverManager/SQLExtendedFetch.c + ${LIBRARY_DIR}/DriverManager/SQLFetch.c + ${LIBRARY_DIR}/DriverManager/SQLFetchScroll.c + ${LIBRARY_DIR}/DriverManager/SQLForeignKeys.c + ${LIBRARY_DIR}/DriverManager/SQLForeignKeysW.c + ${LIBRARY_DIR}/DriverManager/SQLFreeConnect.c + ${LIBRARY_DIR}/DriverManager/SQLFreeEnv.c + ${LIBRARY_DIR}/DriverManager/SQLFreeHandle.c + ${LIBRARY_DIR}/DriverManager/SQLFreeStmt.c + ${LIBRARY_DIR}/DriverManager/SQLGetConnectAttr.c + ${LIBRARY_DIR}/DriverManager/SQLGetConnectAttrW.c + ${LIBRARY_DIR}/DriverManager/SQLGetConnectOption.c + ${LIBRARY_DIR}/DriverManager/SQLGetConnectOptionW.c + ${LIBRARY_DIR}/DriverManager/SQLGetCursorName.c + ${LIBRARY_DIR}/DriverManager/SQLGetCursorNameW.c + ${LIBRARY_DIR}/DriverManager/SQLGetData.c + ${LIBRARY_DIR}/DriverManager/SQLGetDescField.c + ${LIBRARY_DIR}/DriverManager/SQLGetDescFieldW.c + ${LIBRARY_DIR}/DriverManager/SQLGetDescRec.c + ${LIBRARY_DIR}/DriverManager/SQLGetDescRecW.c + ${LIBRARY_DIR}/DriverManager/SQLGetDiagField.c + ${LIBRARY_DIR}/DriverManager/SQLGetDiagFieldW.c + ${LIBRARY_DIR}/DriverManager/SQLGetDiagRec.c + ${LIBRARY_DIR}/DriverManager/SQLGetDiagRecW.c + ${LIBRARY_DIR}/DriverManager/SQLGetEnvAttr.c + ${LIBRARY_DIR}/DriverManager/SQLGetFunctions.c + ${LIBRARY_DIR}/DriverManager/SQLGetInfo.c + ${LIBRARY_DIR}/DriverManager/SQLGetInfoW.c + ${LIBRARY_DIR}/DriverManager/SQLGetStmtAttr.c + ${LIBRARY_DIR}/DriverManager/SQLGetStmtAttrW.c + ${LIBRARY_DIR}/DriverManager/SQLGetStmtOption.c + ${LIBRARY_DIR}/DriverManager/SQLGetTypeInfo.c + ${LIBRARY_DIR}/DriverManager/SQLGetTypeInfoW.c + ${LIBRARY_DIR}/DriverManager/SQLMoreResults.c + ${LIBRARY_DIR}/DriverManager/SQLNativeSql.c + ${LIBRARY_DIR}/DriverManager/SQLNativeSqlW.c + ${LIBRARY_DIR}/DriverManager/SQLNumParams.c + ${LIBRARY_DIR}/DriverManager/SQLNumResultCols.c + ${LIBRARY_DIR}/DriverManager/SQLParamData.c + ${LIBRARY_DIR}/DriverManager/SQLParamOptions.c + ${LIBRARY_DIR}/DriverManager/SQLPrepare.c + ${LIBRARY_DIR}/DriverManager/SQLPrepareW.c + ${LIBRARY_DIR}/DriverManager/SQLPrimaryKeys.c + ${LIBRARY_DIR}/DriverManager/SQLPrimaryKeysW.c + ${LIBRARY_DIR}/DriverManager/SQLProcedureColumns.c + ${LIBRARY_DIR}/DriverManager/SQLProcedureColumnsW.c + ${LIBRARY_DIR}/DriverManager/SQLProcedures.c + ${LIBRARY_DIR}/DriverManager/SQLProceduresW.c + ${LIBRARY_DIR}/DriverManager/SQLPutData.c + ${LIBRARY_DIR}/DriverManager/SQLRowCount.c + ${LIBRARY_DIR}/DriverManager/SQLSetConnectAttr.c + ${LIBRARY_DIR}/DriverManager/SQLSetConnectAttrW.c + ${LIBRARY_DIR}/DriverManager/SQLSetConnectOption.c + ${LIBRARY_DIR}/DriverManager/SQLSetConnectOptionW.c + ${LIBRARY_DIR}/DriverManager/SQLSetCursorName.c + ${LIBRARY_DIR}/DriverManager/SQLSetCursorNameW.c + ${LIBRARY_DIR}/DriverManager/SQLSetDescField.c + ${LIBRARY_DIR}/DriverManager/SQLSetDescFieldW.c + ${LIBRARY_DIR}/DriverManager/SQLSetDescRec.c + ${LIBRARY_DIR}/DriverManager/SQLSetEnvAttr.c + ${LIBRARY_DIR}/DriverManager/SQLSetParam.c + ${LIBRARY_DIR}/DriverManager/SQLSetPos.c + ${LIBRARY_DIR}/DriverManager/SQLSetScrollOptions.c + ${LIBRARY_DIR}/DriverManager/SQLSetStmtAttr.c + ${LIBRARY_DIR}/DriverManager/SQLSetStmtAttrW.c + ${LIBRARY_DIR}/DriverManager/SQLSetStmtOption.c + ${LIBRARY_DIR}/DriverManager/SQLSetStmtOptionW.c + ${LIBRARY_DIR}/DriverManager/SQLSpecialColumns.c + ${LIBRARY_DIR}/DriverManager/SQLSpecialColumnsW.c + ${LIBRARY_DIR}/DriverManager/SQLStatistics.c + ${LIBRARY_DIR}/DriverManager/SQLStatisticsW.c + ${LIBRARY_DIR}/DriverManager/SQLTablePrivileges.c + ${LIBRARY_DIR}/DriverManager/SQLTablePrivilegesW.c + ${LIBRARY_DIR}/DriverManager/SQLTables.c + ${LIBRARY_DIR}/DriverManager/SQLTablesW.c + ${LIBRARY_DIR}/DriverManager/SQLTransact.c + ${LIBRARY_DIR}/ini/_iniDump.c + ${LIBRARY_DIR}/ini/_iniObjectRead.c + ${LIBRARY_DIR}/ini/_iniPropertyRead.c + ${LIBRARY_DIR}/ini/_iniScanUntilObject.c + ${LIBRARY_DIR}/ini/iniAllTrim.c + ${LIBRARY_DIR}/ini/iniAppend.c + ${LIBRARY_DIR}/ini/iniClose.c + ${LIBRARY_DIR}/ini/iniCommit.c + ${LIBRARY_DIR}/ini/iniCursor.c + ${LIBRARY_DIR}/ini/iniDelete.c + ${LIBRARY_DIR}/ini/iniElement.c + ${LIBRARY_DIR}/ini/iniElementCount.c + ${LIBRARY_DIR}/ini/iniGetBookmark.c + ${LIBRARY_DIR}/ini/iniGotoBookmark.c + ${LIBRARY_DIR}/ini/iniObject.c + ${LIBRARY_DIR}/ini/iniObjectDelete.c + ${LIBRARY_DIR}/ini/iniObjectEOL.c + ${LIBRARY_DIR}/ini/iniObjectFirst.c + ${LIBRARY_DIR}/ini/iniObjectInsert.c + ${LIBRARY_DIR}/ini/iniObjectLast.c + ${LIBRARY_DIR}/ini/iniObjectNext.c + ${LIBRARY_DIR}/ini/iniObjectSeek.c + ${LIBRARY_DIR}/ini/iniObjectSeekSure.c + ${LIBRARY_DIR}/ini/iniObjectUpdate.c + ${LIBRARY_DIR}/ini/iniOpen.c + ${LIBRARY_DIR}/ini/iniProperty.c + ${LIBRARY_DIR}/ini/iniPropertyDelete.c + ${LIBRARY_DIR}/ini/iniPropertyEOL.c + ${LIBRARY_DIR}/ini/iniPropertyFirst.c + ${LIBRARY_DIR}/ini/iniPropertyInsert.c + ${LIBRARY_DIR}/ini/iniPropertyLast.c + ${LIBRARY_DIR}/ini/iniPropertyNext.c + ${LIBRARY_DIR}/ini/iniPropertySeek.c + ${LIBRARY_DIR}/ini/iniPropertySeekSure.c + ${LIBRARY_DIR}/ini/iniPropertyUpdate.c + ${LIBRARY_DIR}/ini/iniPropertyValue.c + ${LIBRARY_DIR}/ini/iniToUpper.c + ${LIBRARY_DIR}/ini/iniValue.c + ${LIBRARY_DIR}/log/_logFreeMsg.c + ${LIBRARY_DIR}/log/logClear.c + ${LIBRARY_DIR}/log/logClose.c + ${LIBRARY_DIR}/log/logOn.c + ${LIBRARY_DIR}/log/logOpen.c + ${LIBRARY_DIR}/log/logPeekMsg.c + ${LIBRARY_DIR}/log/logPopMsg.c + ${LIBRARY_DIR}/log/logPushMsg.c + ${LIBRARY_DIR}/lst/_lstAdjustCurrent.c + ${LIBRARY_DIR}/lst/_lstDump.c + ${LIBRARY_DIR}/lst/_lstFreeItem.c + ${LIBRARY_DIR}/lst/_lstNextValidItem.c + ${LIBRARY_DIR}/lst/_lstPrevValidItem.c + ${LIBRARY_DIR}/lst/_lstVisible.c + ${LIBRARY_DIR}/lst/lstAppend.c + ${LIBRARY_DIR}/lst/lstClose.c + ${LIBRARY_DIR}/lst/lstDelete.c + ${LIBRARY_DIR}/lst/lstEOL.c + ${LIBRARY_DIR}/lst/lstFirst.c + ${LIBRARY_DIR}/lst/lstGet.c + ${LIBRARY_DIR}/lst/lstGetBookMark.c + ${LIBRARY_DIR}/lst/lstGoto.c + ${LIBRARY_DIR}/lst/lstGotoBookMark.c + ${LIBRARY_DIR}/lst/lstInsert.c + ${LIBRARY_DIR}/lst/lstLast.c + ${LIBRARY_DIR}/lst/lstNext.c + ${LIBRARY_DIR}/lst/lstOpen.c + ${LIBRARY_DIR}/lst/lstOpenCursor.c + ${LIBRARY_DIR}/lst/lstPrev.c + ${LIBRARY_DIR}/lst/lstSeek.c + ${LIBRARY_DIR}/lst/lstSeekItem.c + ${LIBRARY_DIR}/lst/lstSet.c + ${LIBRARY_DIR}/lst/lstSetFreeFunc.c + ${LIBRARY_DIR}/odbcinst/_logging.c + ${LIBRARY_DIR}/odbcinst/_odbcinst_ConfigModeINI.c + ${LIBRARY_DIR}/odbcinst/_odbcinst_GetEntries.c + ${LIBRARY_DIR}/odbcinst/_odbcinst_GetSections.c + ${LIBRARY_DIR}/odbcinst/_odbcinst_SystemINI.c + ${LIBRARY_DIR}/odbcinst/_odbcinst_UserINI.c + ${LIBRARY_DIR}/odbcinst/_SQLDriverConnectPrompt.c + ${LIBRARY_DIR}/odbcinst/_SQLGetInstalledDrivers.c + ${LIBRARY_DIR}/odbcinst/_SQLWriteInstalledDrivers.c + ${LIBRARY_DIR}/odbcinst/ODBCINSTConstructProperties.c + ${LIBRARY_DIR}/odbcinst/ODBCINSTDestructProperties.c + ${LIBRARY_DIR}/odbcinst/ODBCINSTSetProperty.c + ${LIBRARY_DIR}/odbcinst/ODBCINSTValidateProperties.c + ${LIBRARY_DIR}/odbcinst/ODBCINSTValidateProperty.c + ${LIBRARY_DIR}/odbcinst/SQLConfigDataSource.c + ${LIBRARY_DIR}/odbcinst/SQLConfigDriver.c + ${LIBRARY_DIR}/odbcinst/SQLCreateDataSource.c + ${LIBRARY_DIR}/odbcinst/SQLGetAvailableDrivers.c + ${LIBRARY_DIR}/odbcinst/SQLGetConfigMode.c + ${LIBRARY_DIR}/odbcinst/SQLGetInstalledDrivers.c + ${LIBRARY_DIR}/odbcinst/SQLGetPrivateProfileString.c + ${LIBRARY_DIR}/odbcinst/SQLGetTranslator.c + ${LIBRARY_DIR}/odbcinst/SQLInstallDriverEx.c + ${LIBRARY_DIR}/odbcinst/SQLInstallDriverManager.c + ${LIBRARY_DIR}/odbcinst/SQLInstallerError.c + ${LIBRARY_DIR}/odbcinst/SQLInstallODBC.c + ${LIBRARY_DIR}/odbcinst/SQLInstallTranslatorEx.c + ${LIBRARY_DIR}/odbcinst/SQLManageDataSources.c + ${LIBRARY_DIR}/odbcinst/SQLPostInstallerError.c + ${LIBRARY_DIR}/odbcinst/SQLReadFileDSN.c + ${LIBRARY_DIR}/odbcinst/SQLRemoveDriver.c + ${LIBRARY_DIR}/odbcinst/SQLRemoveDriverManager.c + ${LIBRARY_DIR}/odbcinst/SQLRemoveDSNFromIni.c + ${LIBRARY_DIR}/odbcinst/SQLRemoveTranslator.c + ${LIBRARY_DIR}/odbcinst/SQLSetConfigMode.c + ${LIBRARY_DIR}/odbcinst/SQLValidDSN.c + ${LIBRARY_DIR}/odbcinst/SQLWriteDSNToIni.c + ${LIBRARY_DIR}/odbcinst/SQLWriteFileDSN.c + ${LIBRARY_DIR}/odbcinst/SQLWritePrivateProfileString.c + ) + + add_library (unixodbc ${SRCS}) + + target_link_libraries (unixodbc PRIVATE ltdl) + + # SYSTEM_FILE_PATH was changed to /etc + + target_include_directories (unixodbc + PRIVATE + linux_x86_64/private + PUBLIC + linux_x86_64 + ${LIBRARY_DIR}/include + ) + target_compile_definitions (unixodbc PRIVATE -DHAVE_CONFIG_H) + target_compile_options (unixodbc + PRIVATE + -Wno-dangling-else + -Wno-parentheses + -Wno-misleading-indentation + -Wno-unknown-warning-option + -Wno-reserved-id-macro + -O2 + ) endif () + +target_compile_definitions (unixodbc INTERFACE USE_ODBC=1) + +message (STATUS "Using unixodbc") From 1e04014a71711641adede971620e854974702a70 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Aug 2020 23:10:51 +0300 Subject: [PATCH 210/374] Fix build --- programs/CMakeLists.txt | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 15223af7ede..89220251cda 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -17,7 +17,12 @@ option (ENABLE_CLICKHOUSE_COPIER "Enable clickhouse-copier" ${ENABLE_CLICKHOUSE_ option (ENABLE_CLICKHOUSE_FORMAT "Enable clickhouse-format" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_OBFUSCATOR "Enable clickhouse-obfuscator" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_ODBC_BRIDGE "Enable clickhouse-odbc-bridge" ${ENABLE_CLICKHOUSE_ALL}) -option (ENABLE_CLICKHOUSE_INSTALL "Enable clickhouse-install" ${ENABLE_CLICKHOUSE_ALL}) + +if (CLICKHOUSE_SPLIT_BINARY) + option (ENABLE_CLICKHOUSE_INSTALL "Enable clickhouse-install" OFF) +else () + option (ENABLE_CLICKHOUSE_INSTALL "Enable clickhouse-install" ${ENABLE_CLICKHOUSE_ALL}) +endif () if(NOT (MAKE_STATIC_LIBRARIES OR SPLIT_SHARED_LIBRARIES)) set(CLICKHOUSE_ONE_SHARED 1) From bc5e94a09ba2b7e01075f470c82eda4949442081 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 7 Aug 2020 23:11:05 +0300 Subject: [PATCH 211/374] Add test. --- .../0_stateless/01418_custom_settings.reference | 12 +++++++++++- .../queries/0_stateless/01418_custom_settings.sql | 15 ++++++++++++++- 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01418_custom_settings.reference b/tests/queries/0_stateless/01418_custom_settings.reference index 5f239591218..cf0cb35c72a 100644 --- a/tests/queries/0_stateless/01418_custom_settings.reference +++ b/tests/queries/0_stateless/01418_custom_settings.reference @@ -6,6 +6,7 @@ custom_a UInt64_5 custom_b Int64_-177 custom_c Float64_98.11 custom_d \'abc def\' + changed String \N Nullable(Nothing) 50000 UInt16 @@ -14,7 +15,16 @@ custom_a \'changed\' custom_b NULL custom_c UInt64_50000 custom_d Float64_1.11 -0 UInt8 + +404 UInt16 + +-0.333 Float64 +custom_e Float64_-0.333 +404 UInt16 +custom_e UInt64_404 +word String +custom_f \'word\' +0 test String custom_compound.identifier.v1 \'test\' diff --git a/tests/queries/0_stateless/01418_custom_settings.sql b/tests/queries/0_stateless/01418_custom_settings.sql index 1c87b143b79..95051db3a34 100644 --- a/tests/queries/0_stateless/01418_custom_settings.sql +++ b/tests/queries/0_stateless/01418_custom_settings.sql @@ -8,6 +8,7 @@ SELECT getSetting('custom_c') as v, toTypeName(v); SELECT getSetting('custom_d') as v, toTypeName(v); SELECT name, value FROM system.settings WHERE name LIKE 'custom_%' ORDER BY name; +SELECT ''; SET custom_a = 'changed'; SET custom_b = NULL; SET custom_c = 50000; @@ -18,12 +19,24 @@ SELECT getSetting('custom_c') as v, toTypeName(v); SELECT getSetting('custom_d') as v, toTypeName(v); SELECT name, value FROM system.settings WHERE name LIKE 'custom_%' ORDER BY name; +SELECT ''; SELECT getSetting('custom_e') as v, toTypeName(v); -- { serverError 115 } -- Setting not found. -SET custom_e = 0; +SET custom_e = 404; SELECT getSetting('custom_e') as v, toTypeName(v); SET invalid_custom = 8; -- { serverError 115 } -- Setting is neither a builtin nor started with one of the registered prefixes for user-defined settings. +SELECT ''; +SELECT getSetting('custom_e') as v, toTypeName(v) SETTINGS custom_e = -0.333; +SELECT name, value FROM system.settings WHERE name = 'custom_e' SETTINGS custom_e = -0.333; +SELECT getSetting('custom_e') as v, toTypeName(v); +SELECT name, value FROM system.settings WHERE name = 'custom_e'; + +SELECT getSetting('custom_f') as v, toTypeName(v) SETTINGS custom_f = 'word'; +SELECT name, value FROM system.settings WHERE name = 'custom_f' SETTINGS custom_f = 'word'; +SELECT getSetting('custom_f') as v, toTypeName(v); -- { serverError 115 } -- Setting not found. +SELECT COUNT() FROM system.settings WHERE name = 'custom_f'; + SELECT ''; SET custom_compound.identifier.v1 = 'test'; SELECT getSetting('custom_compound.identifier.v1') as v, toTypeName(v); From 3b9f5a827c5ee1d8c5376a8948ecf97516092978 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 7 Aug 2020 23:49:14 +0300 Subject: [PATCH 212/374] Fix getting settings specified in query. --- src/Functions/getScalar.cpp | 3 ++- src/Interpreters/ActionsVisitor.cpp | 7 +------ 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Functions/getScalar.cpp b/src/Functions/getScalar.cpp index 6cc0f2ffe06..a9cf538000d 100644 --- a/src/Functions/getScalar.cpp +++ b/src/Functions/getScalar.cpp @@ -44,7 +44,8 @@ public: if (arguments.size() != 1 || !isString(arguments[0].type) || !arguments[0].column || !isColumnConst(*arguments[0].column)) throw Exception("Function " + getName() + " accepts one const string argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); auto scalar_name = assert_cast(*arguments[0].column).getValue(); - scalar = context.getScalar(scalar_name).getByPosition(0); + const Context & query_context = context.hasQueryContext() ? context.getQueryContext() : context; + scalar = query_context.getScalar(scalar_name).getByPosition(0); return scalar.type; } diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 41784d5e35c..b382b26dcec 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -586,15 +586,10 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (AggregateFunctionFactory::instance().isAggregateFunctionName(node.name)) return; - /// Context object that we pass to function should live during query. - const Context & function_context = data.context.hasQueryContext() - ? data.context.getQueryContext() - : data.context; - FunctionOverloadResolverPtr function_builder; try { - function_builder = FunctionFactory::instance().get(node.name, function_context); + function_builder = FunctionFactory::instance().get(node.name, data.context); } catch (DB::Exception & e) { From 68145598265413f91fb5bd7713159dacab8dbb29 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 10 Aug 2020 02:09:56 +0300 Subject: [PATCH 213/374] Update aggregatefunction.md --- docs/ru/sql-reference/data-types/aggregatefunction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/aggregatefunction.md b/docs/ru/sql-reference/data-types/aggregatefunction.md index 073e392c8da..1033d09113f 100644 --- a/docs/ru/sql-reference/data-types/aggregatefunction.md +++ b/docs/ru/sql-reference/data-types/aggregatefunction.md @@ -58,6 +58,6 @@ SELECT uniqMerge(state) FROM (SELECT uniqState(UserID) AS state FROM table GROUP ## Пример использования {#primer-ispolzovaniia} -Смотрите в описании движка [AggregatingMergeTree](../../sql-reference/data-types/aggregatefunction.md). +Смотрите в описании движка [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md). [Оригинальная статья](https://clickhouse.tech/docs/ru/data_types/nested_data_structures/aggregatefunction/) From 64352c0dc4c383dfc879e6caa973d1aa37c8b836 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 10 Aug 2020 02:10:17 +0300 Subject: [PATCH 214/374] Fix idiotic data rot (@blinkov) --- docs/ru/sql-reference/data-types/aggregatefunction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/aggregatefunction.md b/docs/ru/sql-reference/data-types/aggregatefunction.md index 1033d09113f..9fbf3dbeded 100644 --- a/docs/ru/sql-reference/data-types/aggregatefunction.md +++ b/docs/ru/sql-reference/data-types/aggregatefunction.md @@ -2,7 +2,7 @@ Промежуточное состояние агрегатной функции. Чтобы его получить, используются агрегатные функции с суффиксом `-State`. Чтобы в дальнейшем получить агрегированные данные необходимо использовать те же агрегатные функции с суффиксом `-Merge`. -`AggregateFunction(name, types\_of\_arguments…)` — параметрический тип данных. +`AggregateFunction(name, types_of_arguments…)` — параметрический тип данных. **Параметры** From 361417bea1cf05564f5cdaa008b411b4ae5a9dcc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Aug 2020 02:18:07 +0300 Subject: [PATCH 215/374] Fix "split" build --- programs/install/CMakeLists.txt | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/programs/install/CMakeLists.txt b/programs/install/CMakeLists.txt index 4b9817e0d37..79f9558ccbb 100644 --- a/programs/install/CMakeLists.txt +++ b/programs/install/CMakeLists.txt @@ -8,5 +8,4 @@ set (CLICKHOUSE_INSTALL_LINK readpassphrase ) -clickhouse_program_add(install) - +clickhouse_program_add_library(install) From c7a6a18a758f8491f4773a990ae60a1ae0f53b73 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Aug 2020 02:58:09 +0300 Subject: [PATCH 216/374] Preparation to enable clickhouse install on Mac and FreeBSD --- CMakeLists.txt | 8 ++++++++ contrib/cctz-cmake/CMakeLists.txt | 2 +- programs/server/CMakeLists.txt | 2 +- 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 6f77118e159..c43b881d482 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -155,6 +155,14 @@ else () message(FATAL_ERROR "Cannot find objcopy.") endif () +if (OS_DARWIN) + set(WHOLE_ARCHIVE -all_load) + set(NO_WHOLE_ARCHIVE -noall_load) +else () + set(WHOLE_ARCHIVE --whole-archive) + set(NO_WHOLE_ARCHIVE --no-whole-archive) +endif () + option (ADD_GDB_INDEX_FOR_GOLD "Set to add .gdb-index to resulting binaries for gold linker. NOOP if lld is used." 0) if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") if (LINKER_NAME STREQUAL "lld") diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 7ca28fc12c3..ee87b3233ad 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -616,7 +616,7 @@ if (USE_INTERNAL_CCTZ) # libraries in linker command. To avoid this we hardcode whole-archive # library into single string. add_dependencies(cctz tzdata) - target_link_libraries(cctz INTERFACE "-Wl,--whole-archive $ -Wl,--no-whole-archive") + target_link_libraries(cctz INTERFACE "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}") endif () else () diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index 339d59f4336..5500a4680b7 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -4,7 +4,7 @@ set(CLICKHOUSE_SERVER_SOURCES ) if (OS_LINUX AND ARCH_AMD64) - set (LINK_CONFIG_LIB INTERFACE "-Wl,--whole-archive $ -Wl,--no-whole-archive") + set (LINK_CONFIG_LIB INTERFACE "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}") endif () set (CLICKHOUSE_SERVER_LINK From 8054a08c5564b5de78abe0f4c3d3309ad410055e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Aug 2020 03:44:21 +0300 Subject: [PATCH 217/374] Fix build after merge --- src/Common/tests/average.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/tests/average.cpp b/src/Common/tests/average.cpp index d277acdb834..0351c0a91a1 100644 --- a/src/Common/tests/average.cpp +++ b/src/Common/tests/average.cpp @@ -9,6 +9,9 @@ #include #include +#if !__clang__ +#pragma GCC diagnostic ignored "-Wframe-larger-than=" +#endif /** This test program evaluates different solutions for a simple degenerate task: * Aggregate data by UInt8 key, calculate "avg" function on Float values. From 84584659b68d5c0b26df8926bcdbdce24a8dac29 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Aug 2020 04:06:06 +0300 Subject: [PATCH 218/374] Better error messages --- src/Interpreters/loadMetadata.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 0bd97252090..712e98a4692 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -70,8 +70,16 @@ static void loadDatabase( database_attach_query = "CREATE DATABASE " + backQuoteIfNeed(database); } - executeCreateQuery(database_attach_query, context, database, - database_metadata_file, force_restore_data); + try + { + executeCreateQuery(database_attach_query, context, database, + database_metadata_file, force_restore_data); + } + catch (Exception & e) + { + e.addMessage(fmt::format("while loading database {} from file {}", backQuote(database), database_path)); + throw; + } } From 90b762849636412a5f4b48e9d995ad44a88417c6 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 9 Aug 2020 22:55:58 +0800 Subject: [PATCH 219/374] ISSUES-4006 try fix test failure & add some test --- src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp | 7 ++++++- .../materialize_with_ddl.py | 4 ++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp b/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp index 8ac781b533a..b2b40ddaba2 100644 --- a/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp +++ b/src/Parsers/MySQL/tests/gtest_alter_command_parser.cpp @@ -161,7 +161,12 @@ TEST(ParserAlterCommand, RenameAlterCommand) ast = tryParserQuery(alter_p, "RENAME TO new_table_name"); EXPECT_EQ(ast->as()->type, ASTAlterCommand::RENAME_TABLE); - EXPECT_EQ(ast->as()->index_name, "new_table_name"); + EXPECT_EQ(ast->as()->new_table_name, "new_table_name"); + + ast = tryParserQuery(alter_p, "RENAME TO new_database_name.new_table_name"); + EXPECT_EQ(ast->as()->type, ASTAlterCommand::RENAME_TABLE); + EXPECT_EQ(ast->as()->new_table_name, "new_table_name"); + EXPECT_EQ(ast->as()->new_database_name, "new_database_name"); } TEST(ParserAlterCommand, ModifyAlterCommand) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index d2516c03572..dd3586a4624 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -301,7 +301,7 @@ def alter_rename_table_with_materialize_mysql_database(clickhouse_node, mysql_no mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") - mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column, RENAME TO test_database.test_table_2, RENAME TO test_table_3") + mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column, RENAME TO test_database.test_table_2, RENAME TO test_database.test_table_3") # create mapping clickhouse_node.query( @@ -313,7 +313,7 @@ def alter_rename_table_with_materialize_mysql_database(clickhouse_node, mysql_no mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_3\n") check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\ndrop_column\tInt32\t\t\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column, RENAME TO test_table_2, RENAME TO test_database.test_table_4") + mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column, RENAME TO test_database.test_table_2, RENAME TO test_database.test_table_4") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_3\ntest_table_4\n") check_query(clickhouse_node, "DESC test_database.test_table_4 FORMAT TSV", "id\tInt32\t\t\t\t\t\n") From 0c15f3b6c8a50f5bdac8e9cad1d112a5aff96d9b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Aug 2020 05:58:08 +0300 Subject: [PATCH 220/374] Allow server to startup if there are leftovers from unsuccessfull database creations --- src/Databases/DatabaseOrdinary.cpp | 1 - src/Interpreters/InterpreterCreateQuery.cpp | 1 - src/Interpreters/loadMetadata.cpp | 20 +++++++++++++++++++- 3 files changed, 19 insertions(+), 3 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 613a3bee66a..8ff06f04f91 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -142,7 +142,6 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto } }; - iterateMetadataFiles(context, process_metadata); size_t total_tables = file_names.size() - total_dictionaries; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a0b5c51f323..6cd9eddbf56 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -100,7 +100,6 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) throw Exception("Database " + database_name + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS); } - /// Will write file with database metadata, if needed. String database_name_escaped = escapeForFileName(database_name); fs::path metadata_path = fs::canonical(context.getPath()); diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 712e98a4692..779a3d21ffc 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -77,7 +77,7 @@ static void loadDatabase( } catch (Exception & e) { - e.addMessage(fmt::format("while loading database {} from file {}", backQuote(database), database_path)); + e.addMessage(fmt::format("while loading database {} from path {}", backQuote(database), database_path)); throw; } } @@ -88,6 +88,8 @@ static void loadDatabase( void loadMetadata(Context & context, const String & default_database_name) { + Poco::Logger * log = &Poco::Logger::get("loadMetadata"); + String path = context.getPath() + "metadata"; /** There may exist 'force_restore_data' file, that means, @@ -114,6 +116,22 @@ void loadMetadata(Context & context, const String & default_database_name) if (db_name != SYSTEM_DATABASE) databases.emplace(unescapeForFileName(db_name), path + "/" + db_name); } + + /// Temporary fails may be left from previous server runs. + if (endsWith(it.name(), ".tmp")) + { + LOG_WARNING(log, "Removing temporary file {}", it->path()); + try + { + it->remove(); + } + catch (...) + { + /// It does not prevent server to startup. + tryLogCurrentException(log); + } + } + continue; } From 09d30a3699f2d715d11e083012a552c6a90e3e73 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Aug 2020 06:26:23 +0300 Subject: [PATCH 221/374] Attempt to fix "Arcadia" build --- programs/ya.make | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/programs/ya.make b/programs/ya.make index fac3d128033..31e7b6f8569 100644 --- a/programs/ya.make +++ b/programs/ya.make @@ -6,6 +6,10 @@ CFLAGS( -DENABLE_CLICKHOUSE_SERVER ) +ADDINCL ( + GLOBAL clickhouse/base/readpassphrase +) + PEERDIR( clickhouse/base/daemon clickhouse/base/loggers From 4c5d7cd733af6e4113d6ec15823bf6cc7ec1a78f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 10 Aug 2020 09:16:14 +0300 Subject: [PATCH 222/374] Fix data corruption @blinkov --- docs/ru/whats-new/extended-roadmap.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/ru/whats-new/extended-roadmap.md b/docs/ru/whats-new/extended-roadmap.md index e5a46c2da25..d222fd4e284 100644 --- a/docs/ru/whats-new/extended-roadmap.md +++ b/docs/ru/whats-new/extended-roadmap.md @@ -8,7 +8,7 @@ ### 1.1. Индексы по z-Order curve, normalized z-Order curve {#indeksy-po-z-order-curve-normalized-z-order-curve} -[\#6286](https://github.com/ClickHouse/ClickHouse/pull/6286) +[#6286](https://github.com/ClickHouse/ClickHouse/pull/6286) Задача также относится к категории «17. Работа с географическими данными», так как geoHash - это частный случай z-Order curve. Также связана с задачей 24.27 для нечёткого поиска полудубликатов строк, так как позволит индексировать min-hash. @@ -23,7 +23,7 @@ Q2. Делает [Александр Токмаков](https://github.com/tavplu Манипуляции с каталогом баз данных: запросы CREATE TABLE, DROP TABLE, RENAME TABLE и DATABASE, требуют синхронизации с помощью блокировок. Эта синхронизация становится весьма сложной, так как на неё полагается много внутренних структур данных. -Предлагается реализовать альтернативный подход, в котором таблицы и базы данных являются всего лишь ссылками на persistent объекты. Подробное описание задачи: [\#6787](https://github.com/ClickHouse/ClickHouse/issues/6787) +Предлагается реализовать альтернативный подход, в котором таблицы и базы данных являются всего лишь ссылками на persistent объекты. Подробное описание задачи: [#6787](https://github.com/ClickHouse/ClickHouse/issues/6787) Upd. Сделана крупная часть задачи, но ориентироваться стоит уже на Q2. Upd. Pull request готов для мержа. @@ -81,7 +81,7 @@ Q1. Закоммичено, но есть технический долг, ко Будет делать Сорокин Николай, ВШЭ и Яндекс. Upd. Есть pull request. Upd. Сделано. -Сейчас пользователь может задать в таблице выражение, которое определяет, сколько времени хранятся данные. Обычно это выражение задаётся относительно значения столбца с датой - например: удалять данные через три месяца. https://clickhouse.tech/docs/ru/operations/table_engines/mergetree/\#table_engine-mergetree-ttl +Сейчас пользователь может задать в таблице выражение, которое определяет, сколько времени хранятся данные. Обычно это выражение задаётся относительно значения столбца с датой - например: удалять данные через три месяца. https://clickhouse.tech/docs/ru/operations/table_engines/mergetree/#table_engine-mergetree-ttl Это может быть задано для всей таблицы (тогда строки целиком удаляются после указанного времени) или для отдельных столбцов (тогда данные столбца физически удаляются с диска, а строки в таблице остаются; при чтении значений столбца, они читаются как значения по-умолчанию). @@ -232,7 +232,7 @@ Upd. В очереди. Иван Лежанкин. ### 2.12. Декларативные сигнатуры функций {#deklarativnye-signatury-funktsii} -[\#3775](https://github.com/ClickHouse/ClickHouse/pull/3775) +[#3775](https://github.com/ClickHouse/ClickHouse/pull/3775) Задачу делает Алексей Миловидов. Прогресс 50% и разработка временно приостановлена. @@ -318,7 +318,7 @@ Upd. Сейчас обсуждается, как сделать другую з ### 5.1. + Разделение задач на более мелкие куски в clickhouse-copier {#razdelenie-zadach-na-bolee-melkie-kuski-v-clickhouse-copier} -[\#9075](https://github.com/ClickHouse/ClickHouse/pull/9075) +[#9075](https://github.com/ClickHouse/ClickHouse/pull/9075) Q1. Нужно для Метрики, в очереди. Никита Михайлов. Upd. Задача на финальной стадии разработки. @@ -350,7 +350,7 @@ Upd. Появилась вторая версия LTS - 20.3. ### 6.2. + Добавление memory profiler {#dobavlenie-memory-profiler} -[\#6387](https://github.com/ClickHouse/ClickHouse/issues/6387) +[#6387](https://github.com/ClickHouse/ClickHouse/issues/6387) Сравнительно простая задача, но только для опытных разработчиков. Нужна всем. Иван Лежанкин. Q1. Сделано. @@ -525,7 +525,7 @@ Upd. Добавлены прямые ссылки и инструкция в д ### 7.19. + Доделать (проверить) автосборку под AArch64 {#dodelat-proverit-avtosborku-pod-aarch64} -https://github.com/ClickHouse/ClickHouse/issues/8027\#issuecomment-566670282 +https://github.com/ClickHouse/ClickHouse/issues/8027#issuecomment-566670282 Проверили на настоящем сервере Huawei, а также в специальном Docker контейнере, который содержит внутри qemu-user-static. Также можно проверить на Cavium, на Raspberry Pi а также на твоём Android телефоне. From 3696fcbb1f7ab7ce5a07c18c56472d5c5dfe7d00 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 10 Aug 2020 13:14:31 +0300 Subject: [PATCH 223/374] changes --- CHANGELOG.md | 164 ++++++++++++++++++++++++++------------------------- 1 file changed, 83 insertions(+), 81 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a8783f2042f..16e3d3a5935 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,136 +1,138 @@ - ## ClickHouse release 20.6 -## ClickHouse release v20.6.2.15-prestable FIXME as compared to v20.5.4.40-stable +### ClickHouse release v20.6.3.28-stable #### New Feature -* Implementation of PostgreSQL-like ILIKE operator for https://github.com/ClickHouse/ClickHouse/issues/11710. [#12125](https://github.com/ClickHouse/ClickHouse/pull/12125) ([Mike](https://github.com/myrrc)). -* Support RIGHT and FULL JOIN with `set join_algorithm=partial_merge`. Only ALL strictness is supported (ANY, SEMI, ANTI, ASOF are not). [#12118](https://github.com/ClickHouse/ClickHouse/pull/12118) ([Artem Zuikov](https://github.com/4ertus2)). -* Add a function initializedAggregation to initialize an aggregation based on a single value. [#12109](https://github.com/ClickHouse/ClickHouse/pull/12109) ([Guillaume Tassery](https://github.com/YiuRULE)). -* #4006 Support ALTER TABLE ... [ADD|MODIFY] COLUMN ... FIRST. [#12073](https://github.com/ClickHouse/ClickHouse/pull/12073) ([Winter Zhang](https://github.com/zhang2014)). -* add function `parseDateTimeBestEffortUS`. [#12028](https://github.com/ClickHouse/ClickHouse/pull/12028) ([flynn](https://github.com/ucasFL)). -* Switched paths in S3 metadata to relative which allows to handle S3 blobs more easily. [#11892](https://github.com/ClickHouse/ClickHouse/pull/11892) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Initial implementation of `EXPLAIN` query. Syntax: `EXPLAIN SELECT ...`. This fixes [#1118](https://github.com/ClickHouse/ClickHouse/issues/1118). [#11873](https://github.com/ClickHouse/ClickHouse/pull/11873) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Add ORCBlockOutputFormat. [#11662](https://github.com/ClickHouse/ClickHouse/pull/11662) ([Kruglov Pavel](https://github.com/Avogar)). -* Add storage RabbitMQ. [#11069](https://github.com/ClickHouse/ClickHouse/pull/11069) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Added new in-memory format of parts in `MergeTree`-family tables, which stores data in memory. Parts are written on disk at first merge. Part will be created in in-memory format if its size in rows or bytes is below thresholds `min_rows_for_compact_part` and `min_bytes_for_compact_part`. Also optional support of Write-Ahead-Log is available, which is enabled by default and is controlled by setting `in_memory_parts_enable_wal`. [#10697](https://github.com/ClickHouse/ClickHouse/pull/10697) ([Anton Popov](https://github.com/CurtizJ)). +* Implemented PostgreSQL-like `ILIKE` operator for [#11710](https://github.com/ClickHouse/ClickHouse/issues/11710). [#12125](https://github.com/ClickHouse/ClickHouse/pull/12125) ([Mike](https://github.com/myrrc)). +* Supported RIGHT and FULL JOIN with `set join_algorithm=partial_merge`. Only ALL strictness is allowed (ANY, SEMI, ANTI, ASOF are not). [#12118](https://github.com/ClickHouse/ClickHouse/pull/12118) ([Artem Zuikov](https://github.com/4ertus2)). +* Added a function `initializedAggregation` to initialize an aggregation based on a single value. [#12109](https://github.com/ClickHouse/ClickHouse/pull/12109) ([Guillaume Tassery](https://github.com/YiuRULE)). +* Supported `ALTER TABLE ... [ADD|MODIFY] COLUMN ... FIRST` [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#12073](https://github.com/ClickHouse/ClickHouse/pull/12073) ([Winter Zhang](https://github.com/zhang2014)). +* Added function `parseDateTimeBestEffortUS`. [#12028](https://github.com/ClickHouse/ClickHouse/pull/12028) ([flynn](https://github.com/ucasFL)). +* Added an initial implementation of `EXPLAIN` query. Syntax: `EXPLAIN SELECT ...`. This fixes [#1118](https://github.com/ClickHouse/ClickHouse/issues/1118). [#11873](https://github.com/ClickHouse/ClickHouse/pull/11873) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Added `ORCBlockOutputFormat`. [#11662](https://github.com/ClickHouse/ClickHouse/pull/11662) ([Kruglov Pavel](https://github.com/Avogar)). +* Added storage `RabbitMQ`. [#11069](https://github.com/ClickHouse/ClickHouse/pull/11069) ([Kseniia Sumarokova](https://github.com/kssenii)). #### Bug Fix -* Fix wrong index analysis with functions. It could lead to pruning wrong parts, while reading from `MergeTree` tables. Fixes [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). -* Fix error `Cannot convert column because it is constant but values of constants are different in source and result` for remote queries which use deterministic functions in scope of query, but not deterministic between queries, like `now()`, `now64()`, `randConstant()`. Fixes [#11327](https://github.com/ClickHouse/ClickHouse/issues/11327). [#13075](https://github.com/ClickHouse/ClickHouse/pull/13075) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix rare bug when `ALTER DELETE` and `ALTER MODIFY COLUMN` queries executed simultaneously as a single mutation. Bug leads to an incorrect amount of rows in `count.txt` and as a consequence incorrect data in part. Also, fix a small bug with simultaneous `ALTER RENAME COLUMN` and `ALTER ADD COLUMN`. [#12760](https://github.com/ClickHouse/ClickHouse/pull/12760) ([alesapin](https://github.com/alesapin)). -* Fix CAST(Nullable(String), Enum()). [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) ([Azat Khuzhin](https://github.com/azat)). -* Fix performance with large tuples, which are interpreted as functions in `IN` section. The case when user write `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). -* Fix memory tracking for input_format_parallel_parsing (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). -* fixes [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572) fix bloom filter index with const expression. [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). -* Fix SIGSEGV in StorageKafka when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). -* Add support for function `if` with `Array(UUID)` arguments. This fixes [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `aggregate function any(x) is found inside another aggregate function in query` error with `SET optimize_move_functions_out_of_any = 1` and aliases inside `any()`. [#13419](https://github.com/ClickHouse/ClickHouse/pull/13419) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixed `PrettyCompactMonoBlock` for clickhouse-local. Fixed extremes/totals with `PrettyCompactMonoBlock`. This fixes [#7746](https://github.com/ClickHouse/ClickHouse/issues/7746). [#13394](https://github.com/ClickHouse/ClickHouse/pull/13394) ([Azat Khuzhin](https://github.com/azat)). +* Fixed possible error `Totals having transform was already added to pipeline` in case of a query from delayed replica. [#13290](https://github.com/ClickHouse/ClickHouse/pull/13290) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* The server may crash if user passed specifically crafted arguments to the function `h3ToChildren`. This fixes [#13275](https://github.com/ClickHouse/ClickHouse/issues/13275). [#13277](https://github.com/ClickHouse/ClickHouse/pull/13277) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potentially low performance and slightly incorrect result for `uniqExact`, `topK`, `sumDistinct` and similar aggregate functions called on Float types with NaN values. It also triggered assert in debug build. This fixes [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed function if with nullable constexpr as cond that is not literal NULL. Fixes [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed assert in `arrayElement` function in case of array elements are Nullable and array subscript is also Nullable. This fixes [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `DateTime64` conversion functions with constant argument. [#13205](https://github.com/ClickHouse/ClickHouse/pull/13205) ([Azat Khuzhin](https://github.com/azat)). +* Fixed wrong index analysis with functions. It could lead to pruning wrong parts, while reading from `MergeTree` tables. Fixes [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed error `Cannot convert column because it is constant but values of constants are different in source and result` for remote queries which use deterministic functions in scope of query, but not deterministic between queries, like `now()`, `now64()`, `randConstant()`. Fixes [#11327](https://github.com/ClickHouse/ClickHouse/issues/11327). [#13075](https://github.com/ClickHouse/ClickHouse/pull/13075) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed rare bug when `ALTER DELETE` and `ALTER MODIFY COLUMN` queries executed simultaneously as a single mutation. Bug leads to an incorrect amount of rows in `count.txt` and as a consequence incorrect data in part. Also, fix a small bug with simultaneous `ALTER RENAME COLUMN` and `ALTER ADD COLUMN`. [#12760](https://github.com/ClickHouse/ClickHouse/pull/12760) ([alesapin](https://github.com/alesapin)). +* Fixed `CAST(Nullable(String), Enum())`. [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) ([Azat Khuzhin](https://github.com/azat)). +* Fixed a performance with large tuples, which are interpreted as functions in `IN` section. The case when user write `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed memory tracking for `input_format_parallel_parsing` (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). +* Fixed bloom filter index with const expression. This fixes [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572). [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `SIGSEGV` in `StorageKafka` when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). +* Added support for function `if` with `Array(UUID)` arguments. This fixes [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* `CREATE USER IF NOT EXISTS` now doesn't throw exception if the user exists. This fixes [#12507](https://github.com/ClickHouse/ClickHouse/issues/12507). [#12646](https://github.com/ClickHouse/ClickHouse/pull/12646) ([Vitaly Baranov](https://github.com/vitlibar)). * Better exception message in disk access storage. [#12625](https://github.com/ClickHouse/ClickHouse/pull/12625) ([alesapin](https://github.com/alesapin)). -* Fix lack of aliases with function `any`. [#12593](https://github.com/ClickHouse/ClickHouse/pull/12593) ([Anton Popov](https://github.com/CurtizJ)). -* Fix race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). +* The function `groupArrayMoving*` was not working for distributed queries. It's result was calculated within incorrect data type (without promotion to the largest type). The function `groupArrayMovingAvg` was returning integer number that was inconsistent with the `avg` function. This fixes [#12568](https://github.com/ClickHouse/ClickHouse/issues/12568). [#12622](https://github.com/ClickHouse/ClickHouse/pull/12622) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed lack of aliases with function `any`. [#12593](https://github.com/ClickHouse/ClickHouse/pull/12593) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). * Remove data for Distributed tables (blocks from async INSERTs) on DROP TABLE. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). -* Fix bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). +* Fixed bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). * Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). * Fixing race condition in live view tables which could cause data duplication. [#12519](https://github.com/ClickHouse/ClickHouse/pull/12519) ([vzakaznikov](https://github.com/vzakaznikov)). * Fixed performance issue, while reading from compact parts. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). -* Fix backwards compatibility in binary format of `AggregateFunction(avg, ...)` values. This fixes [#12342](https://github.com/ClickHouse/ClickHouse/issues/12342). [#12486](https://github.com/ClickHouse/ClickHouse/pull/12486) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix SETTINGS parse after FORMAT. [#12480](https://github.com/ClickHouse/ClickHouse/pull/12480) ([Azat Khuzhin](https://github.com/azat)). -* SystemLog: do not write to ordinary server log under mutex. This can lead to deadlock if `text_log` is enabled. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix overflow when very large LIMIT or OFFSET is specified. This fixes [#10470](https://github.com/ClickHouse/ClickHouse/issues/10470). This fixes [#11372](https://github.com/ClickHouse/ClickHouse/issues/11372). [#12427](https://github.com/ClickHouse/ClickHouse/pull/12427) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed possible segfault if StorageMerge. Closes [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). -* Reverts change introduced in [#11079](https://github.com/ClickHouse/ClickHouse/issues/11079) to resolve [#12098](https://github.com/ClickHouse/ClickHouse/issues/12098). [#12397](https://github.com/ClickHouse/ClickHouse/pull/12397) ([Mike](https://github.com/myrrc)). +* Fixed backwards compatibility in binary format of `AggregateFunction(avg, ...)` values. This fixes [#12342](https://github.com/ClickHouse/ClickHouse/issues/12342). [#12486](https://github.com/ClickHouse/ClickHouse/pull/12486) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed SETTINGS parse after FORMAT. [#12480](https://github.com/ClickHouse/ClickHouse/pull/12480) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the deadlock if `text_log` is enabled. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed overflow when very large `LIMIT` or `OFFSET` is specified. This fixes [#10470](https://github.com/ClickHouse/ClickHouse/issues/10470). This fixes [#11372](https://github.com/ClickHouse/ClickHouse/issues/11372). [#12427](https://github.com/ClickHouse/ClickHouse/pull/12427) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed possible segfault if `StorageMerge`. This fixes [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). +* Reverted change introduced in [#11079](https://github.com/ClickHouse/ClickHouse/issues/11079) to resolve [#12098](https://github.com/ClickHouse/ClickHouse/issues/12098). [#12397](https://github.com/ClickHouse/ClickHouse/pull/12397) ([Mike](https://github.com/myrrc)). * Additional check for arguments of bloom filter index. This fixes [#11408](https://github.com/ClickHouse/ClickHouse/issues/11408). [#12388](https://github.com/ClickHouse/ClickHouse/pull/12388) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Avoid exception when negative or floating point constant is used in WHERE condition for indexed tables. This fixes [#11905](https://github.com/ClickHouse/ClickHouse/issues/11905). [#12384](https://github.com/ClickHouse/ClickHouse/pull/12384) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allow to CLEAR column even if there are depending DEFAULT expressions. This fixes [#12333](https://github.com/ClickHouse/ClickHouse/issues/12333). [#12378](https://github.com/ClickHouse/ClickHouse/pull/12378) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix TOTALS/ROLLUP/CUBE for aggregate functions with `-State` and `Nullable` arguments. This fixes [#12163](https://github.com/ClickHouse/ClickHouse/issues/12163). [#12376](https://github.com/ClickHouse/ClickHouse/pull/12376) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix error message and exit codes for `ALTER RENAME COLUMN` queries, when `RENAME` is not allowed. Fixes [#12301](https://github.com/ClickHouse/ClickHouse/issues/12301) and [#12303](https://github.com/ClickHouse/ClickHouse/issues/12303). [#12335](https://github.com/ClickHouse/ClickHouse/pull/12335) ([alesapin](https://github.com/alesapin)). -* Fix very rare race condition in ReplicatedMergeTreeQueue. [#12315](https://github.com/ClickHouse/ClickHouse/pull/12315) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allowed to `CLEAR` column even if there are depending `DEFAULT` expressions. This fixes [#12333](https://github.com/ClickHouse/ClickHouse/issues/12333). [#12378](https://github.com/ClickHouse/ClickHouse/pull/12378) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `TOTALS/ROLLUP/CUBE` for aggregate functions with `-State` and `Nullable` arguments. This fixes [#12163](https://github.com/ClickHouse/ClickHouse/issues/12163). [#12376](https://github.com/ClickHouse/ClickHouse/pull/12376) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed error message and exit codes for `ALTER RENAME COLUMN` queries, when `RENAME` is not allowed. Fixes [#12301](https://github.com/ClickHouse/ClickHouse/issues/12301) and [#12303](https://github.com/ClickHouse/ClickHouse/issues/12303). [#12335](https://github.com/ClickHouse/ClickHouse/pull/12335) ([alesapin](https://github.com/alesapin)). +* Fixed very rare race condition in `ReplicatedMergeTreeQueue`. [#12315](https://github.com/ClickHouse/ClickHouse/pull/12315) ([alexey-milovidov](https://github.com/alexey-milovidov)). * When using codec `Delta` or `DoubleDelta` with non fixed width types, exception with code `LOGICAL_ERROR` was returned instead of exception with code `BAD_ARGUMENTS` (we ensure that exceptions with code logical error never happen). This fixes [#12110](https://github.com/ClickHouse/ClickHouse/issues/12110). [#12308](https://github.com/ClickHouse/ClickHouse/pull/12308) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix order of columns in `WITH FILL` modifier. Previously order of columns of `ORDER BY` statement wasn't respected. [#12306](https://github.com/ClickHouse/ClickHouse/pull/12306) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed order of columns in `WITH FILL` modifier. Previously order of columns of `ORDER BY` statement wasn't respected. [#12306](https://github.com/ClickHouse/ClickHouse/pull/12306) ([Anton Popov](https://github.com/CurtizJ)). * Avoid "bad cast" exception when there is an expression that filters data by virtual columns (like `_table` in `Merge` tables) or by "index" columns in system tables such as filtering by database name when querying from `system.tables`, and this expression returns `Nullable` type. This fixes [#12166](https://github.com/ClickHouse/ClickHouse/issues/12166). [#12305](https://github.com/ClickHouse/ClickHouse/pull/12305) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix TTL after renaming column, on which depends TTL expression. [#12304](https://github.com/ClickHouse/ClickHouse/pull/12304) ([Anton Popov](https://github.com/CurtizJ)). -* kafka: fix SIGSEGV if there is an message with error in the middle of the batch. [#12302](https://github.com/ClickHouse/ClickHouse/pull/12302) ([Azat Khuzhin](https://github.com/azat)). -* Some threads might randomly hang for a few seconds during DNS cache updating. It's fixed. [#12296](https://github.com/ClickHouse/ClickHouse/pull/12296) ([tavplubix](https://github.com/tavplubix)). -* Fix typo in setting name. [#12292](https://github.com/ClickHouse/ClickHouse/pull/12292) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Show error after TrieDictionary failed to load. [#12290](https://github.com/ClickHouse/ClickHouse/pull/12290) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed `TTL` after renaming column, on which depends TTL expression. [#12304](https://github.com/ClickHouse/ClickHouse/pull/12304) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed SIGSEGV if there is an message with error in the middle of the batch in `Kafka` Engine. [#12302](https://github.com/ClickHouse/ClickHouse/pull/12302) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the situation when some threads might randomly hang for a few seconds during `DNS` cache updating. [#12296](https://github.com/ClickHouse/ClickHouse/pull/12296) ([tavplubix](https://github.com/tavplubix)). +* Fixed typo in setting name. [#12292](https://github.com/ClickHouse/ClickHouse/pull/12292) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Show error after `TrieDictionary` failed to load. [#12290](https://github.com/ClickHouse/ClickHouse/pull/12290) ([Vitaly Baranov](https://github.com/vitlibar)). * The function `arrayFill` worked incorrectly for empty arrays that may lead to crash. This fixes [#12263](https://github.com/ClickHouse/ClickHouse/issues/12263). [#12279](https://github.com/ClickHouse/ClickHouse/pull/12279) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Implement conversions to the common type for LowCardinality types. This allows to execute UNION ALL of tables with columns of LowCardinality and other columns. This fixes [#8212](https://github.com/ClickHouse/ClickHouse/issues/8212). This fixes [#4342](https://github.com/ClickHouse/ClickHouse/issues/4342). [#12275](https://github.com/ClickHouse/ClickHouse/pull/12275) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Not for changelog. Cherry-pick after [#12196](https://github.com/ClickHouse/ClickHouse/issues/12196). [#12271](https://github.com/ClickHouse/ClickHouse/pull/12271) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed behaviour on reaching redirect limit in request to S3 storage. [#12256](https://github.com/ClickHouse/ClickHouse/pull/12256) ([ianton-ru](https://github.com/ianton-ru)). +* Implement conversions to the common type for `LowCardinality` types. This allows to execute UNION ALL of tables with columns of LowCardinality and other columns. This fixes [#8212](https://github.com/ClickHouse/ClickHouse/issues/8212). This fixes [#4342](https://github.com/ClickHouse/ClickHouse/issues/4342). [#12275](https://github.com/ClickHouse/ClickHouse/pull/12275) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the behaviour on reaching redirect limit in request to `S3` storage. [#12256](https://github.com/ClickHouse/ClickHouse/pull/12256) ([ianton-ru](https://github.com/ianton-ru)). * Fixed the behaviour when during multiple sequential inserts in `StorageFile` header for some special types was written more than once. This fixed [#6155](https://github.com/ClickHouse/ClickHouse/issues/6155). [#12197](https://github.com/ClickHouse/ClickHouse/pull/12197) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Fixed logical functions for UInt8 values when they are not equal to 0 or 1. [#12196](https://github.com/ClickHouse/ClickHouse/pull/12196) ([Alexander Kazakov](https://github.com/Akazz)). * Cap max_memory_usage* limits to the process resident memory. [#12182](https://github.com/ClickHouse/ClickHouse/pull/12182) ([Azat Khuzhin](https://github.com/azat)). -* Fix dictGet arguments check during GROUP BY injective functions elimination. [#12179](https://github.com/ClickHouse/ClickHouse/pull/12179) ([Azat Khuzhin](https://github.com/azat)). +* Fix dictGet arguments check during `GROUP BY` injective functions elimination. [#12179](https://github.com/ClickHouse/ClickHouse/pull/12179) ([Azat Khuzhin](https://github.com/azat)). * Fixed the behaviour when `SummingMergeTree` engine sums up columns from partition key. Added an exception in case of explicit definition of columns to sum which intersects with partition key columns. This fixes [#7867](https://github.com/ClickHouse/ClickHouse/issues/7867). [#12173](https://github.com/ClickHouse/ClickHouse/pull/12173) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Don't split the dictionary source's table name into schema and table name itself if ODBC connection doesn't support schema. [#12165](https://github.com/ClickHouse/ClickHouse/pull/12165) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix wrong logic in ALTER DELETE that leads to deleting of records when condition evaluates to NULL. This fixes [#9088](https://github.com/ClickHouse/ClickHouse/issues/9088). This closes [#12106](https://github.com/ClickHouse/ClickHouse/issues/12106). [#12153](https://github.com/ClickHouse/ClickHouse/pull/12153) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix transform of query to send to external DBMS (e.g. MySQL, ODBC) in presense of aliases. This fixes [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix bad code in redundant ORDER BY optimization. The bug was introduced in [#10067](https://github.com/ClickHouse/ClickHouse/issues/10067). [#12148](https://github.com/ClickHouse/ClickHouse/pull/12148) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix potential overflow in integer division. This fixes [#12119](https://github.com/ClickHouse/ClickHouse/issues/12119). [#12140](https://github.com/ClickHouse/ClickHouse/pull/12140) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix potential infinite loop in `greatCircleDistance`, `geoDistance`. This fixes [#12117](https://github.com/ClickHouse/ClickHouse/issues/12117). [#12137](https://github.com/ClickHouse/ClickHouse/pull/12137) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed wrong logic in `ALTER DELETE` that leads to deleting of records when condition evaluates to NULL. This fixes [#9088](https://github.com/ClickHouse/ClickHouse/issues/9088). This closes [#12106](https://github.com/ClickHouse/ClickHouse/issues/12106). [#12153](https://github.com/ClickHouse/ClickHouse/pull/12153) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed transform of query to send to external DBMS (e.g. MySQL, ODBC) in presense of aliases. This fixes [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed bad code in redundant ORDER BY optimization. The bug was introduced in [#10067](https://github.com/ClickHouse/ClickHouse/issues/10067). [#12148](https://github.com/ClickHouse/ClickHouse/pull/12148) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential overflow in integer division. This fixes [#12119](https://github.com/ClickHouse/ClickHouse/issues/12119). [#12140](https://github.com/ClickHouse/ClickHouse/pull/12140) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential infinite loop in `greatCircleDistance`, `geoDistance`. This fixes [#12117](https://github.com/ClickHouse/ClickHouse/issues/12117). [#12137](https://github.com/ClickHouse/ClickHouse/pull/12137) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Normalize "pid" file handling. In previous versions the server may refuse to start if it was killed without proper shutdown and if there is another process that has the same pid as previously runned server. Also pid file may be removed in unsuccessful server startup even if there is another server running. This fixes [#3501](https://github.com/ClickHouse/ClickHouse/issues/3501). [#12133](https://github.com/ClickHouse/ClickHouse/pull/12133) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix bug which leads to incorrect table metadata in ZooKeepeer for ReplicatedVersionedCollapsingMergeTree tables. Fixes [#12093](https://github.com/ClickHouse/ClickHouse/issues/12093). [#12121](https://github.com/ClickHouse/ClickHouse/pull/12121) ([alesapin](https://github.com/alesapin)). +* Fixed bug which leads to incorrect table metadata in ZooKeepeer for ReplicatedVersionedCollapsingMergeTree tables. Fixes [#12093](https://github.com/ClickHouse/ClickHouse/issues/12093). [#12121](https://github.com/ClickHouse/ClickHouse/pull/12121) ([alesapin](https://github.com/alesapin)). * Avoid "There is no query" exception for materialized views with joins or with subqueries attached to system logs (system.query_log, metric_log, etc) or to engine=Buffer underlying table. [#12120](https://github.com/ClickHouse/ClickHouse/pull/12120) ([filimonov](https://github.com/filimonov)). -* Fix handling dependency of table with ENGINE=Dictionary on dictionary. This fixes [#10994](https://github.com/ClickHouse/ClickHouse/issues/10994). This fixes [#10397](https://github.com/ClickHouse/ClickHouse/issues/10397). [#12116](https://github.com/ClickHouse/ClickHouse/pull/12116) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed handling dependency of table with ENGINE=Dictionary on dictionary. This fixes [#10994](https://github.com/ClickHouse/ClickHouse/issues/10994). This fixes [#10397](https://github.com/ClickHouse/ClickHouse/issues/10397). [#12116](https://github.com/ClickHouse/ClickHouse/pull/12116) ([Vitaly Baranov](https://github.com/vitlibar)). * Format `Parquet` now properly works with `LowCardinality` and `LowCardinality(Nullable)` types. Fixes [#12086](https://github.com/ClickHouse/ClickHouse/issues/12086), [#8406](https://github.com/ClickHouse/ClickHouse/issues/8406). [#12108](https://github.com/ClickHouse/ClickHouse/pull/12108) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix performance for selects with `UNION` caused by wrong limit for the total number of threads. Fixes [#12030](https://github.com/ClickHouse/ClickHouse/issues/12030). [#12103](https://github.com/ClickHouse/ClickHouse/pull/12103) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix segfault with `-StateResample` combinators. [#12092](https://github.com/ClickHouse/ClickHouse/pull/12092) ([Anton Popov](https://github.com/CurtizJ)). -* Fix empty `result_rows` and `result_bytes` metrics in `system.quey_log` for selects. Fixes [#11595](https://github.com/ClickHouse/ClickHouse/issues/11595). [#12089](https://github.com/ClickHouse/ClickHouse/pull/12089) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix unnecessary limiting the number of threads for selects from `VIEW`. Fixes [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix SIGSEGV in StorageKafka on DROP TABLE. [#12075](https://github.com/ClickHouse/ClickHouse/pull/12075) ([Azat Khuzhin](https://github.com/azat)). -* Fix possible crash while using wrong type for `PREWHERE`. Fixes [#12053](https://github.com/ClickHouse/ClickHouse/issues/12053), [#12060](https://github.com/ClickHouse/ClickHouse/issues/12060). [#12060](https://github.com/ClickHouse/ClickHouse/pull/12060) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix error `Cannot capture column` for higher-order functions with `Tuple(LowCardinality)` argument. Fixes [#9766](https://github.com/ClickHouse/ClickHouse/issues/9766). [#12055](https://github.com/ClickHouse/ClickHouse/pull/12055) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix constraints check if constraint is a constant expression. This fixes [#11360](https://github.com/ClickHouse/ClickHouse/issues/11360). [#12042](https://github.com/ClickHouse/ClickHouse/pull/12042) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix wrong result and potential crash when invoking function `if` with arguments of type `FixedString` with different sizes. This fixes [#11362](https://github.com/ClickHouse/ClickHouse/issues/11362). [#12021](https://github.com/ClickHouse/ClickHouse/pull/12021) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed performance for selects with `UNION` caused by wrong limit for the total number of threads. Fixes [#12030](https://github.com/ClickHouse/ClickHouse/issues/12030). [#12103](https://github.com/ClickHouse/ClickHouse/pull/12103) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed segfault with `-StateResample` combinators. [#12092](https://github.com/ClickHouse/ClickHouse/pull/12092) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed empty `result_rows` and `result_bytes` metrics in `system.quey_log` for selects. Fixes [#11595](https://github.com/ClickHouse/ClickHouse/issues/11595). [#12089](https://github.com/ClickHouse/ClickHouse/pull/12089) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed unnecessary limiting the number of threads for selects from `VIEW`. Fixes [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed SIGSEGV in StorageKafka on DROP TABLE. [#12075](https://github.com/ClickHouse/ClickHouse/pull/12075) ([Azat Khuzhin](https://github.com/azat)). +* Fixed possible crash while using wrong type for `PREWHERE`. Fixes [#12053](https://github.com/ClickHouse/ClickHouse/issues/12053), [#12060](https://github.com/ClickHouse/ClickHouse/issues/12060). [#12060](https://github.com/ClickHouse/ClickHouse/pull/12060) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed error `Cannot capture column` for higher-order functions with `Tuple(LowCardinality)` argument. Fixes [#9766](https://github.com/ClickHouse/ClickHouse/issues/9766). [#12055](https://github.com/ClickHouse/ClickHouse/pull/12055) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed constraints check if constraint is a constant expression. This fixes [#11360](https://github.com/ClickHouse/ClickHouse/issues/11360). [#12042](https://github.com/ClickHouse/ClickHouse/pull/12042) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed wrong result and potential crash when invoking function `if` with arguments of type `FixedString` with different sizes. This fixes [#11362](https://github.com/ClickHouse/ClickHouse/issues/11362). [#12021](https://github.com/ClickHouse/ClickHouse/pull/12021) ([alexey-milovidov](https://github.com/alexey-milovidov)). #### Improvement -* Allow to set JOIN kind and type in more standad way: `LEFT SEMI JOIN` instead of `SEMI LEFT JOIN`. For now both are correct. [#12520](https://github.com/ClickHouse/ClickHouse/pull/12520) ([Artem Zuikov](https://github.com/4ertus2)). +* Allowed to set `JOIN` kind and type in more standart way: `LEFT SEMI JOIN` instead of `SEMI LEFT JOIN`. For now both are correct. [#12520](https://github.com/ClickHouse/ClickHouse/pull/12520) ([Artem Zuikov](https://github.com/4ertus2)). * lifetime_rows/lifetime_bytes for Buffer engine. [#12421](https://github.com/ClickHouse/ClickHouse/pull/12421) ([Azat Khuzhin](https://github.com/azat)). * Write the detail exception message to the client instead of 'MySQL server has gone away'. [#12383](https://github.com/ClickHouse/ClickHouse/pull/12383) ([BohuTANG](https://github.com/BohuTANG)). -* This setting allows to chose charset for printing grids (either utf8 or ascii). [#12372](https://github.com/ClickHouse/ClickHouse/pull/12372) ([Sabyanin Maxim](https://github.com/s-mx)). -* 1. Support MySQL 'SELECT DATABASE()' [#9336](https://github.com/ClickHouse/ClickHouse/issues/9336) 2. Add MySQL replacement query integration test. [#12314](https://github.com/ClickHouse/ClickHouse/pull/12314) ([BohuTANG](https://github.com/BohuTANG)). -* Add `KILL QUERY [connection_id]` for the MySQL client/driver to cancel the long query, issue [#12038](https://github.com/ClickHouse/ClickHouse/issues/12038). [#12152](https://github.com/ClickHouse/ClickHouse/pull/12152) ([BohuTANG](https://github.com/BohuTANG)). +* Allows to change a charset which is used for printing grids borders. Available charsets are following: UTF-8, ASCII. Setting `output_format_pretty_grid_charset` enables this feature. [#12372](https://github.com/ClickHouse/ClickHouse/pull/12372) ([Sabyanin Maxim](https://github.com/s-mx)). +* Supported MySQL 'SELECT DATABASE()' [#9336](https://github.com/ClickHouse/ClickHouse/issues/9336) 2. Add MySQL replacement query integration test. [#12314](https://github.com/ClickHouse/ClickHouse/pull/12314) ([BohuTANG](https://github.com/BohuTANG)). +* Added `KILL QUERY [connection_id]` for the MySQL client/driver to cancel the long query, issue [#12038](https://github.com/ClickHouse/ClickHouse/issues/12038). [#12152](https://github.com/ClickHouse/ClickHouse/pull/12152) ([BohuTANG](https://github.com/BohuTANG)). * Added support for `%g` (two digit ISO year) and `%G` (four digit ISO year) substitutions in `formatDateTime` function. [#12136](https://github.com/ClickHouse/ClickHouse/pull/12136) ([vivarum](https://github.com/vivarum)). -* Add 'type' column in system.disks. [#12115](https://github.com/ClickHouse/ClickHouse/pull/12115) ([ianton-ru](https://github.com/ianton-ru)). -* Improves `REVOKE` command: now it requires grant/admin option for only access which will be revoked. For example, to execute `REVOKE ALL ON *.* FROM user1` now it doesn't require to have full access rights granted with grant option. Added command `REVOKE ALL FROM user1` - it revokes all granted roles from `user1`. [#12083](https://github.com/ClickHouse/ClickHouse/pull/12083) ([Vitaly Baranov](https://github.com/vitlibar)). +* Added 'type' column in system.disks. [#12115](https://github.com/ClickHouse/ClickHouse/pull/12115) ([ianton-ru](https://github.com/ianton-ru)). +* Improved `REVOKE` command: now it requires grant/admin option for only access which will be revoked. For example, to execute `REVOKE ALL ON *.* FROM user1` now it doesn't require to have full access rights granted with grant option. Added command `REVOKE ALL FROM user1` - it revokes all granted roles from `user1`. [#12083](https://github.com/ClickHouse/ClickHouse/pull/12083) ([Vitaly Baranov](https://github.com/vitlibar)). * Implemented single part uploads for DiskS3. [#12026](https://github.com/ClickHouse/ClickHouse/pull/12026) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Add replica priority for load_balancing (for manual prioritization of the load balancing). [#11995](https://github.com/ClickHouse/ClickHouse/pull/11995) ([Azat Khuzhin](https://github.com/azat)). -* Improved performace of 'ORDER BY' and 'GROUP BY' by prefix of sorting key. [#11696](https://github.com/ClickHouse/ClickHouse/pull/11696) ([Anton Popov](https://github.com/CurtizJ)). +* Added replica priority for load_balancing (for manual prioritization of the load balancing). [#11995](https://github.com/ClickHouse/ClickHouse/pull/11995) ([Azat Khuzhin](https://github.com/azat)). +* Switched paths in S3 metadata to relative which allows to handle S3 blobs more easily. [#11892](https://github.com/ClickHouse/ClickHouse/pull/11892) ([Vladimir Chebotarev](https://github.com/excitoon)). #### Performance Improvement -* Remove injective functions inside `uniq*()` if `set optimize_injective_functions_inside_uniq=1`. [#12337](https://github.com/ClickHouse/ClickHouse/pull/12337) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix "[#10574](https://github.com/ClickHouse/ClickHouse/issues/10574) Index not used for IN operator with literals", performance regression introduced around v19.3. [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). +* Removed injective functions inside `uniq*()` if `set optimize_injective_functions_inside_uniq=1`. [#12337](https://github.com/ClickHouse/ClickHouse/pull/12337) ([Ruslan Kamalov](https://github.com/kamalov-ruslan)). +* Index not used for IN operator with literals", performance regression introduced around v19.3. This fixes "[#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). +* Improved performace of 'ORDER BY' and 'GROUP BY' by prefix of sorting key. [#11696](https://github.com/ClickHouse/ClickHouse/pull/11696) ([Anton Popov](https://github.com/CurtizJ)). + +#### Experimental Feature +* Added new in-memory format of parts in `MergeTree`-family tables, which stores data in memory. Parts are written on disk at first merge. Part will be created in in-memory format if its size in rows or bytes is below thresholds `min_rows_for_compact_part` and `min_bytes_for_compact_part`. Also optional support of Write-Ahead-Log is available, which is enabled by default and is controlled by setting `in_memory_parts_enable_wal`. [#10697](https://github.com/ClickHouse/ClickHouse/pull/10697) ([Anton Popov](https://github.com/CurtizJ)). #### Build/Testing/Packaging Improvement +* Implement AST-based query fuzzing mode for clickhouse-client. See [this label](https://github.com/ClickHouse/ClickHouse/issues?q=label%3Afuzz+is%3Aissue) for the list of issues we recently found by fuzzing. Most of them were found by this tool, and a couple by SQLancer and `00746_sql_fuzzy.pl`. [#12111](https://github.com/ClickHouse/ClickHouse/pull/12111) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Add new type of tests based on Testflows framework. [#12090](https://github.com/ClickHouse/ClickHouse/pull/12090) ([vzakaznikov](https://github.com/vzakaznikov)). * Added S3 HTTPS integration test. [#12412](https://github.com/ClickHouse/ClickHouse/pull/12412) ([Pavel Kovalenko](https://github.com/Jokser)). * Log sanitizer trap messages from separate thread. This will prevent possible deadlock under thread sanitizer. [#12313](https://github.com/ClickHouse/ClickHouse/pull/12313) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Now functional and stress tests will be able to run with old version of `clickhouse-test` script. [#12287](https://github.com/ClickHouse/ClickHouse/pull/12287) ([alesapin](https://github.com/alesapin)). * Remove strange file creation during build in `orc`. [#12258](https://github.com/ClickHouse/ClickHouse/pull/12258) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Remove verbosity from the binary builds. [#12174](https://github.com/ClickHouse/ClickHouse/pull/12174) ([alesapin](https://github.com/alesapin)). * Place common docker compose files to integration docker container. [#12168](https://github.com/ClickHouse/ClickHouse/pull/12168) ([Ilya Yatsishin](https://github.com/qoega)). -* Record additional detail on Dockerfile scan reports. [#12159](https://github.com/ClickHouse/ClickHouse/pull/12159) ([Ivan Blinkov](https://github.com/blinkov)). * Fix warnings from CodeQL. `CodeQL` is another static analyzer that we will use along with `clang-tidy` and `PVS-Studio` that we use already. [#12138](https://github.com/ClickHouse/ClickHouse/pull/12138) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Missed `` is required for `std::atomic<>`. [#12134](https://github.com/ClickHouse/ClickHouse/pull/12134) ([Matwey V. Kornilov](https://github.com/matwey)). * Minor CMake fixes for UNBUNDLED build. [#12131](https://github.com/ClickHouse/ClickHouse/pull/12131) ([Matwey V. Kornilov](https://github.com/matwey)). * Added a showcase of the minimal Docker image without using any Linux distribution. [#12126](https://github.com/ClickHouse/ClickHouse/pull/12126) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Perform an upgrade of system packages in the `clickhouse-server` docker image. [#12124](https://github.com/ClickHouse/ClickHouse/pull/12124) ([Ivan Blinkov](https://github.com/blinkov)). -* Implement AST-based query fuzzing mode for clickhouse-client. See [this label](https://github.com/ClickHouse/ClickHouse/issues?q=label%3Afuzz+is%3Aissue) for the list of issues we recently found by fuzzing. Most of them were found by this tool, and a couple by SQLancer and `00746_sql_fuzzy.pl`. [#12111](https://github.com/ClickHouse/ClickHouse/pull/12111) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Add `UNBUNDLED` flag to `system.build_options` table. Move skip lists for `clickhouse-test` to clickhouse repo. [#12107](https://github.com/ClickHouse/ClickHouse/pull/12107) ([alesapin](https://github.com/alesapin)). * Regular check by [Anchore Container Analysis](https://docs.anchore.com) security analysis tool that looks for [CVE](https://cve.mitre.org/) in `clickhouse-server` Docker image. Also confirms that `Dockerfile` is buildable. Runs daily on `master` and on pull-requests to `Dockerfile`. [#12102](https://github.com/ClickHouse/ClickHouse/pull/12102) ([Ivan Blinkov](https://github.com/blinkov)). * Daily check by [GitHub CodeQL](https://securitylab.github.com/tools/codeql) security analysis tool that looks for [CWE](https://cwe.mitre.org/). [#12101](https://github.com/ClickHouse/ClickHouse/pull/12101) ([Ivan Blinkov](https://github.com/blinkov)). * Install `ca-certificates` before the first `apt-get update` in Dockerfile. [#12095](https://github.com/ClickHouse/ClickHouse/pull/12095) ([Ivan Blinkov](https://github.com/blinkov)). -* Add new type of tests based on Testflows framework. [#12090](https://github.com/ClickHouse/ClickHouse/pull/12090) ([vzakaznikov](https://github.com/vzakaznikov)). - -#### NO CL CATEGORY - -* ... [#12431](https://github.com/ClickHouse/ClickHouse/pull/12431) ([Tom Bombadil](https://github.com/ithangzhou)). -* * Not for changelog. [#12265](https://github.com/ClickHouse/ClickHouse/pull/12265) ([Alexander Kuzmenkov](https://github.com/akuzm)). ## ClickHouse release 20.5 From c8d39996939970342f758dda96464101f58fe8fe Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Mon, 10 Aug 2020 14:18:57 +0300 Subject: [PATCH 224/374] Enable Poco::NetSSL support in ya.make --- base/common/ya.make | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/base/common/ya.make b/base/common/ya.make index caa0d067ef2..fc5de5efab3 100644 --- a/base/common/ya.make +++ b/base/common/ya.make @@ -11,6 +11,7 @@ CFLAGS (GLOBAL -DARCADIA_BUILD) CFLAGS (GLOBAL -DUSE_CPUID=1) CFLAGS (GLOBAL -DUSE_JEMALLOC=0) CFLAGS (GLOBAL -DUSE_RAPIDJSON=1) +CFLAGS (GLOBAL -DUSE_SSL=1) IF (OS_DARWIN) CFLAGS (GLOBAL -DOS_DARWIN) @@ -24,6 +25,7 @@ PEERDIR( contrib/libs/cctz/src contrib/libs/cxxsupp/libcxx-filesystem contrib/libs/poco/Net + contrib/libs/poco/NetSSL_OpenSSL contrib/libs/poco/Util contrib/libs/fmt contrib/restricted/boost @@ -50,7 +52,6 @@ SRCS( shift10.cpp sleep.cpp terminalColors.cpp - ) END() From bece61b1cc4ab36abc7f2c5049d60ed089f8d9ed Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Mon, 10 Aug 2020 14:40:31 +0300 Subject: [PATCH 225/374] Fix codestyle. --- src/Disks/Executor.h | 4 ++-- src/Disks/IDisk.cpp | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Disks/Executor.h b/src/Disks/Executor.h index c47201c402d..61afb569903 100644 --- a/src/Disks/Executor.h +++ b/src/Disks/Executor.h @@ -1,7 +1,7 @@ #pragma once -#include "future" -#include "functional" +#include +#include namespace DB { diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 9a225e8da9e..fd46f693c31 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -6,7 +6,6 @@ #include #include #include -#include "future" namespace DB { From 62054856b72044e09789656d14b6cbf597ca9efd Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 10 Aug 2020 16:10:48 +0300 Subject: [PATCH 226/374] DOCSUP-1999: Restruct sql statements section in separate files. (#13389) * DOCSUP-1999: First commit to test and build docs. * DOCSUP-1999: Move topics from misc.md to single files. * DOCSUP-1999: Add footer original article (for some files) * DOCSUP-1999: Fix some links. * DOCSUP-1999: Add Original article footer for all added files. * DOCSUP-1999: Fix broken link. * DOCSUP-1999: Fix header in misc.md (by pr comment). Fixed footer (original article). * DOCSUP-1999: Fix broken links for build. Co-authored-by: romanzhukov --- docs/ru/sql-reference/statements/alter.md | 2 +- docs/ru/sql-reference/statements/attach.md | 23 ++ .../sql-reference/statements/check-table.md | 44 +++ .../statements/describe-table.md | 24 ++ docs/ru/sql-reference/statements/detach.md | 19 + docs/ru/sql-reference/statements/drop.md | 84 ++++ docs/ru/sql-reference/statements/exists.md | 15 + docs/ru/sql-reference/statements/grant.md | 6 +- docs/ru/sql-reference/statements/kill.md | 73 ++++ docs/ru/sql-reference/statements/misc.md | 365 +----------------- docs/ru/sql-reference/statements/optimize.md | 25 ++ docs/ru/sql-reference/statements/rename.md | 17 + docs/ru/sql-reference/statements/set-role.md | 57 +++ docs/ru/sql-reference/statements/set.md | 22 ++ docs/ru/sql-reference/statements/truncate.md | 17 + docs/ru/sql-reference/statements/use.md | 16 + 16 files changed, 460 insertions(+), 349 deletions(-) create mode 100644 docs/ru/sql-reference/statements/attach.md create mode 100644 docs/ru/sql-reference/statements/check-table.md create mode 100644 docs/ru/sql-reference/statements/describe-table.md create mode 100644 docs/ru/sql-reference/statements/detach.md create mode 100644 docs/ru/sql-reference/statements/drop.md create mode 100644 docs/ru/sql-reference/statements/exists.md create mode 100644 docs/ru/sql-reference/statements/kill.md create mode 100644 docs/ru/sql-reference/statements/optimize.md create mode 100644 docs/ru/sql-reference/statements/rename.md create mode 100644 docs/ru/sql-reference/statements/set-role.md create mode 100644 docs/ru/sql-reference/statements/set.md create mode 100644 docs/ru/sql-reference/statements/truncate.md create mode 100644 docs/ru/sql-reference/statements/use.md diff --git a/docs/ru/sql-reference/statements/alter.md b/docs/ru/sql-reference/statements/alter.md index e923f9c5682..7cac767e842 100644 --- a/docs/ru/sql-reference/statements/alter.md +++ b/docs/ru/sql-reference/statements/alter.md @@ -493,7 +493,7 @@ ALTER TABLE [db.]table MATERIALIZE INDEX name IN PARTITION partition_name Мутации линейно упорядочены между собой и накладываются на каждый кусок в порядке добавления. Мутации также упорядочены со вставками - гарантируется, что данные, вставленные в таблицу до начала выполнения запроса мутации, будут изменены, а данные, вставленные после окончания запроса мутации, изменены не будут. При этом мутации никак не блокируют вставки. -Запрос завершается немедленно после добавления информации о мутации (для реплицированных таблиц - в ZooKeeper, для нереплицированных - на файловую систему). Сама мутация выполняется асинхронно, используя настройки системного профиля. Следить за ходом её выполнения можно по таблице [`system.mutations`](../../operations/system-tables/mutations.md#system_tables-mutations). Добавленные мутации будут выполняться до конца даже в случае перезапуска серверов ClickHouse. Откатить мутацию после её добавления нельзя, но если мутация по какой-то причине не может выполниться до конца, её можно остановить с помощью запроса [`KILL MUTATION`](misc.md#kill-mutation-statement). +Запрос завершается немедленно после добавления информации о мутации (для реплицированных таблиц - в ZooKeeper, для нереплицированных - на файловую систему). Сама мутация выполняется асинхронно, используя настройки системного профиля. Следить за ходом её выполнения можно по таблице [`system.mutations`](../../operations/system-tables/mutations.md#system_tables-mutations). Добавленные мутации будут выполняться до конца даже в случае перезапуска серверов ClickHouse. Откатить мутацию после её добавления нельзя, но если мутация по какой-то причине не может выполниться до конца, её можно остановить с помощью запроса [`KILL MUTATION`](../../sql-reference/statements/kill.md#kill-mutation). Записи о последних выполненных мутациях удаляются не сразу (количество сохраняемых мутаций определяется параметром движка таблиц `finished_mutations_to_keep`). Более старые записи удаляются. diff --git a/docs/ru/sql-reference/statements/attach.md b/docs/ru/sql-reference/statements/attach.md new file mode 100644 index 00000000000..2aaf133130b --- /dev/null +++ b/docs/ru/sql-reference/statements/attach.md @@ -0,0 +1,23 @@ +--- +toc_priority: 42 +toc_title: ATTACH +--- + +# ATTACH Statement {#attach} + +Запрос полностью аналогичен запросу `CREATE`, но: + +- вместо слова `CREATE` используется слово `ATTACH`; +- запрос не создаёт данные на диске, а предполагает, что данные уже лежат в соответствующих местах, и всего лишь добавляет информацию о таблице на сервер. После выполнения запроса `ATTACH` сервер будет знать о существовании таблицы. + +Если таблица перед этим была отсоединена (`DETACH`), т.е. её структура известна, можно использовать сокращенную форму записи без определения структуры. + +``` sql +ATTACH TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] +``` + +Этот запрос используется при старте сервера. Сервер хранит метаданные таблиц в виде файлов с запросами `ATTACH`, которые он просто исполняет при запуске (за исключением системных таблиц, которые явно создаются на сервере). + + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/attach/) + diff --git a/docs/ru/sql-reference/statements/check-table.md b/docs/ru/sql-reference/statements/check-table.md new file mode 100644 index 00000000000..178b2fc3a48 --- /dev/null +++ b/docs/ru/sql-reference/statements/check-table.md @@ -0,0 +1,44 @@ +--- +toc_priority: 43 +toc_title: CHECK +--- + +# CHECK TABLE Statement {#check-table} + +Проверяет таблицу на повреждение данных. + +``` sql +CHECK TABLE [db.]name +``` + +Запрос `CHECK TABLE` сравнивает текущие размеры файлов (в которых хранятся данные из колонок) с ожидаемыми значениями. Если значения не совпадают, данные в таблице считаются поврежденными. Искажение возможно, например, из-за сбоя при записи данных. + +Ответ содержит колонку `result`, содержащую одну строку с типом [Boolean](../../sql-reference/data-types/boolean.md). Допустимые значения: + +- 0 - данные в таблице повреждены; +- 1 - данные не повреждены. + +Запрос `CHECK TABLE` поддерживает следующие движки таблиц: + +- [Log](../../engines/table-engines/log-family/log.md) +- [TinyLog](../../engines/table-engines/log-family/tinylog.md) +- [StripeLog](../../engines/table-engines/log-family/stripelog.md) +- [Семейство MergeTree](../../engines/table-engines/mergetree-family/index.md) + +При попытке выполнить запрос с таблицами с другими табличными движками, ClickHouse генерирует исключение. + +В движках `*Log` не предусмотрено автоматическое восстановление данных после сбоя. Используйте запрос `CHECK TABLE`, чтобы своевременно выявлять повреждение данных. + +Для движков из семейства `MergeTree` запрос `CHECK TABLE` показывает статус проверки для каждого отдельного куска данных таблицы на локальном сервере. + +**Что делать, если данные повреждены** + +В этом случае можно скопировать оставшиеся неповрежденные данные в другую таблицу. Для этого: + +1. Создайте новую таблицу с такой же структурой, как у поврежденной таблицы. Для этого выполните запрос `CREATE TABLE AS `. +2. Установите значение параметра [max\_threads](../../operations/settings/settings.md#settings-max_threads) в 1. Это нужно для того, чтобы выполнить следующий запрос в одном потоке. Установить значение параметра можно через запрос: `SET max_threads = 1`. +3. Выполните запрос `INSERT INTO SELECT * FROM `. В результате неповрежденные данные будут скопированы в другую таблицу. Обратите внимание, будут скопированы только те данные, которые следуют до поврежденного участка. +4. Перезапустите `clickhouse-client`, чтобы вернуть предыдущее значение параметра `max_threads`. + + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/check-table/) \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/describe-table.md b/docs/ru/sql-reference/statements/describe-table.md new file mode 100644 index 00000000000..5b38dca5833 --- /dev/null +++ b/docs/ru/sql-reference/statements/describe-table.md @@ -0,0 +1,24 @@ +--- +toc_priority: 44 +toc_title: DESCRIBE +--- + +# DESCRIBE TABLE Statement {#misc-describe-table} + +``` sql +DESC|DESCRIBE TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] +``` + +Возвращает описание столбцов таблицы. + +Результат запроса содержит столбцы (все столбцы имеют тип String): + +- `name` — имя столбца таблицы; +- `type`— тип столбца; +- `default_type` — в каком виде задано [выражение для значения по умолчанию](../../sql-reference/statements/create/table.md#create-default-values): `DEFAULT`, `MATERIALIZED` или `ALIAS`. Столбец содержит пустую строку, если значение по умолчанию не задано. +- `default_expression` — значение, заданное в секции `DEFAULT`; +- `comment_expression` — комментарий к столбцу. + +Вложенные структуры данных выводятся в «развёрнутом» виде. То есть, каждый столбец - по отдельности, с именем через точку. + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/describe-table/) diff --git a/docs/ru/sql-reference/statements/detach.md b/docs/ru/sql-reference/statements/detach.md new file mode 100644 index 00000000000..172e86179c8 --- /dev/null +++ b/docs/ru/sql-reference/statements/detach.md @@ -0,0 +1,19 @@ +--- +toc_priority: 45 +toc_title: DETACH +--- + +# DETACH {#detach-statement} + +Удаляет из сервера информацию о таблице name. Сервер перестаёт знать о существовании таблицы. + +``` sql +DETACH TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] +``` + +Но ни данные, ни метаданные таблицы не удаляются. При следующем запуске сервера, сервер прочитает метаданные и снова узнает о таблице. +Также, «отцепленную» таблицу можно прицепить заново запросом `ATTACH` (за исключением системных таблиц, для которых метаданные не хранятся). + +Запроса `DETACH DATABASE` нет. + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/detach/) diff --git a/docs/ru/sql-reference/statements/drop.md b/docs/ru/sql-reference/statements/drop.md new file mode 100644 index 00000000000..4bfd53b1d47 --- /dev/null +++ b/docs/ru/sql-reference/statements/drop.md @@ -0,0 +1,84 @@ +--- +toc_priority: 46 +toc_title: DROP +--- + +# DROP {#drop} + +Запрос имеет два вида: `DROP DATABASE` и `DROP TABLE`. + +``` sql +DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] +``` + +``` sql +DROP [TEMPORARY] TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] +``` + +Удаляет таблицу. +Если указано `IF EXISTS` - не выдавать ошибку, если таблица не существует или база данных не существует. + +## DROP USER {#drop-user-statement} + +Удаляет пользователя. + +### Синтаксис {#drop-user-syntax} + +```sql +DROP USER [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +``` + + +## DROP ROLE {#drop-role-statement} + +Удаляет роль. + +При удалении роль отзывается у всех объектов системы доступа, которым она присвоена. + +### Синтаксис {#drop-role-syntax} + +```sql +DROP ROLE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +``` + +## DROP ROW POLICY {#drop-row-policy-statement} + +Удаляет политику доступа к строкам. + +При удалении политика отзывается у всех объектов системы доступа, которым она присвоена. + +### Синтаксис {#drop-row-policy-syntax} + +``` sql +DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] [ON CLUSTER cluster_name] +``` + + +## DROP QUOTA {#drop-quota-statement} + +Удаляет квоту. + +При удалении квота отзывается у всех объектов системы доступа, которым она присвоена. + +### Синтаксис {#drop-quota-syntax} + +``` sql +DROP QUOTA [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +``` + + +## DROP SETTINGS PROFILE {#drop-settings-profile-statement} + +Удаляет профиль настроек. + +При удалении профиль отзывается у всех объектов системы доступа, которым он присвоен. + +### Синтаксис {#drop-settings-profile-syntax} + +``` sql +DROP [SETTINGS] PROFILE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +``` + + + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/drop/) \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/exists.md b/docs/ru/sql-reference/statements/exists.md new file mode 100644 index 00000000000..865d23ad622 --- /dev/null +++ b/docs/ru/sql-reference/statements/exists.md @@ -0,0 +1,15 @@ +--- +toc_priority: 47 +toc_title: EXISTS +--- + +# EXISTS {#exists} + +``` sql +EXISTS [TEMPORARY] TABLE [db.]name [INTO OUTFILE filename] [FORMAT format] +``` + +Возвращает один столбец типа `UInt8`, содержащий одно значение - `0`, если таблицы или БД не существует и `1`, если таблица в указанной БД существует. + + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/exists/) \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/grant.md b/docs/ru/sql-reference/statements/grant.md index 3cf5af0254a..043df0f7397 100644 --- a/docs/ru/sql-reference/statements/grant.md +++ b/docs/ru/sql-reference/statements/grant.md @@ -291,7 +291,7 @@ GRANT INSERT(x,y) ON db.table TO john - Привилегия `MODIFY SETTING` позволяет изменять настройки движков таблиц. Не влияет на настройки или конфигурационные параметры сервера. - Операция `ATTACH` требует наличие привилегии [CREATE](#grant-create). - Операция `DETACH` требует наличие привилегии [DROP](#grant-drop). -- Для остановки мутации с помощью [KILL MUTATION](misc.md#kill-mutation-statement), необходима привилегия на выполнение данной мутации. Например, чтобы остановить запрос `ALTER UPDATE`, необходима одна из привилегий: `ALTER UPDATE`, `ALTER TABLE` или `ALTER`. +- Для остановки мутации с помощью [KILL MUTATION](../../sql-reference/statements/kill.md#kill-mutation), необходима привилегия на выполнение данной мутации. Например, чтобы остановить запрос `ALTER UPDATE`, необходима одна из привилегий: `ALTER UPDATE`, `ALTER TABLE` или `ALTER`. ### CREATE {#grant-create} @@ -321,7 +321,7 @@ GRANT INSERT(x,y) ON db.table TO john ### TRUNCATE {#grant-truncate} -Разрешает выполнять запросы [TRUNCATE](misc.md#truncate-statement). +Разрешает выполнять запросы [TRUNCATE](../../sql-reference/statements/truncate.md). Уровень: `TABLE`. @@ -348,7 +348,7 @@ GRANT INSERT(x,y) ON db.table TO john ### KILL QUERY {#grant-kill-query} -Разрешает выполнять запросы [KILL](misc.md#kill-query-statement) в соответствии со следующей иерархией привилегий: +Разрешает выполнять запросы [KILL](../../sql-reference/statements/kill.md#kill-query) в соответствии со следующей иерархией привилегий: Уровень: `GLOBAL`. diff --git a/docs/ru/sql-reference/statements/kill.md b/docs/ru/sql-reference/statements/kill.md new file mode 100644 index 00000000000..4fcfa9a61ce --- /dev/null +++ b/docs/ru/sql-reference/statements/kill.md @@ -0,0 +1,73 @@ +--- +toc_priority: 48 +toc_title: KILL +--- + +# KILL {#kill-statements} + +Существует два вида операторов KILL: KILL QUERY и KILL MUTATION + +## KILL QUERY {#kill-query} + +``` sql +KILL QUERY [ON CLUSTER cluster] + WHERE + [SYNC|ASYNC|TEST] + [FORMAT format] +``` + +Пытается принудительно остановить исполняющиеся в данный момент запросы. +Запросы для принудительной остановки выбираются из таблицы system.processes с помощью условия, указанного в секции `WHERE` запроса `KILL`. + +Примеры + +``` sql +-- Принудительно останавливает все запросы с указанным query_id: +KILL QUERY WHERE query_id='2-857d-4a57-9ee0-327da5d60a90' + +-- Синхронно останавливает все запросы пользователя 'username': +KILL QUERY WHERE user='username' SYNC +``` + +Readonly-пользователи могут останавливать только свои запросы. + +По умолчанию используется асинхронный вариант запроса (`ASYNC`), который не дожидается подтверждения остановки запросов. + +Синхронный вариант (`SYNC`) ожидает остановки всех запросов и построчно выводит информацию о процессах по ходу их остановки. +Ответ содержит колонку `kill_status`, которая может принимать следующие значения: + +1. ‘finished’ - запрос был успешно остановлен; +2. ‘waiting’ - запросу отправлен сигнал завершения, ожидается его остановка; +3. остальные значения описывают причину невозможности остановки запроса. + +Тестовый вариант запроса (`TEST`) только проверяет права пользователя и выводит список запросов для остановки. + +## KILL MUTATION {#kill-mutation} + +``` sql +KILL MUTATION [ON CLUSTER cluster] + WHERE + [TEST] + [FORMAT format] +``` + +Пытается остановить выполняющиеся в данные момент [мутации](alter.md#mutations). Мутации для остановки выбираются из таблицы [`system.mutations`](../../operations/system-tables/mutations.md#system_tables-mutations) с помощью условия, указанного в секции `WHERE` запроса `KILL`. + +Тестовый вариант запроса (`TEST`) только проверяет права пользователя и выводит список запросов для остановки. + +Примеры: + +``` sql +-- Останавливает все мутации одной таблицы: +KILL MUTATION WHERE database = 'default' AND table = 'table' + +-- Останавливает конкретную мутацию: +KILL MUTATION WHERE database = 'default' AND table = 'table' AND mutation_id = 'mutation_3.txt' +``` + +Запрос полезен в случаях, когда мутация не может выполниться до конца (например, если функция в запросе мутации бросает исключение на данных таблицы). + +Данные, уже изменённые мутацией, остаются в таблице (отката на старую версию данных не происходит). + + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/kill/) \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/misc.md b/docs/ru/sql-reference/statements/misc.md index 09055de8fdc..e9ceece8b2c 100644 --- a/docs/ru/sql-reference/statements/misc.md +++ b/docs/ru/sql-reference/statements/misc.md @@ -1,347 +1,22 @@ +--- +toc_hidden: true +toc_priority: 41 +--- + # Прочие виды запросов {#prochie-vidy-zaprosov} -## ATTACH {#attach} - -Запрос полностью аналогичен запросу `CREATE`, но: - -- вместо слова `CREATE` используется слово `ATTACH`; -- запрос не создаёт данные на диске, а предполагает, что данные уже лежат в соответствующих местах, и всего лишь добавляет информацию о таблице на сервер. После выполнения запроса `ATTACH` сервер будет знать о существовании таблицы. - -Если таблица перед этим была отсоединена (`DETACH`), т.е. её структура известна, можно использовать сокращенную форму записи без определения структуры. - -``` sql -ATTACH TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] -``` - -Этот запрос используется при старте сервера. Сервер хранит метаданные таблиц в виде файлов с запросами `ATTACH`, которые он просто исполняет при запуске (за исключением системных таблиц, которые явно создаются на сервере). - -## CHECK TABLE {#check-table} - -Проверяет таблицу на повреждение данных. - -``` sql -CHECK TABLE [db.]name -``` - -Запрос `CHECK TABLE` сравнивает текущие размеры файлов (в которых хранятся данные из колонок) с ожидаемыми значениями. Если значения не совпадают, данные в таблице считаются поврежденными. Искажение возможно, например, из-за сбоя при записи данных. - -Ответ содержит колонку `result`, содержащую одну строку с типом [Boolean](../../sql-reference/data-types/boolean.md). Допустимые значения: - -- 0 - данные в таблице повреждены; -- 1 - данные не повреждены. - -Запрос `CHECK TABLE` поддерживает следующие движки таблиц: - -- [Log](../../engines/table-engines/log-family/log.md) -- [TinyLog](../../engines/table-engines/log-family/tinylog.md) -- [StripeLog](../../engines/table-engines/log-family/stripelog.md) -- [Семейство MergeTree](../../engines/table-engines/mergetree-family/index.md) - -При попытке выполнить запрос с таблицами с другими табличными движками, ClickHouse генерирует исключение. - -В движках `*Log` не предусмотрено автоматическое восстановление данных после сбоя. Используйте запрос `CHECK TABLE`, чтобы своевременно выявлять повреждение данных. - -Для движков из семейства `MergeTree` запрос `CHECK TABLE` показывает статус проверки для каждого отдельного куска данных таблицы на локальном сервере. - -**Что делать, если данные повреждены** - -В этом случае можно скопировать оставшиеся неповрежденные данные в другую таблицу. Для этого: - -1. Создайте новую таблицу с такой же структурой, как у поврежденной таблицы. Для этого выполните запрос `CREATE TABLE AS `. -2. Установите значение параметра [max\_threads](../../operations/settings/settings.md#settings-max_threads) в 1. Это нужно для того, чтобы выполнить следующий запрос в одном потоке. Установить значение параметра можно через запрос: `SET max_threads = 1`. -3. Выполните запрос `INSERT INTO SELECT * FROM `. В результате неповрежденные данные будут скопированы в другую таблицу. Обратите внимание, будут скопированы только те данные, которые следуют до поврежденного участка. -4. Перезапустите `clickhouse-client`, чтобы вернуть предыдущее значение параметра `max_threads`. - -## DESCRIBE TABLE {#misc-describe-table} - -``` sql -DESC|DESCRIBE TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] -``` - -Возвращает описание столбцов таблицы. - -Результат запроса содержит столбцы (все столбцы имеют тип String): - -- `name` — имя столбца таблицы; -- `type`— тип столбца; -- `default_type` — в каком виде задано [выражение для значения по умолчанию](create/table.md#create-default-values): `DEFAULT`, `MATERIALIZED` или `ALIAS`. Столбец содержит пустую строку, если значение по умолчанию не задано. -- `default_expression` — значение, заданное в секции `DEFAULT`; -- `comment_expression` — комментарий к столбцу. - -Вложенные структуры данных выводятся в «развёрнутом» виде. То есть, каждый столбец - по отдельности, с именем через точку. - -## DETACH {#detach-statement} - -Удаляет из сервера информацию о таблице name. Сервер перестаёт знать о существовании таблицы. - -``` sql -DETACH TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] -``` - -Но ни данные, ни метаданные таблицы не удаляются. При следующем запуске сервера, сервер прочитает метаданные и снова узнает о таблице. -Также, «отцепленную» таблицу можно прицепить заново запросом `ATTACH` (за исключением системных таблиц, для которых метаданные не хранятся). - -Запроса `DETACH DATABASE` нет. - -## DROP {#drop} - -Запрос имеет два вида: `DROP DATABASE` и `DROP TABLE`. - -``` sql -DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] -``` - -``` sql -DROP [TEMPORARY] TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] -``` - -Удаляет таблицу. -Если указано `IF EXISTS` - не выдавать ошибку, если таблица не существует или база данных не существует. - -## DROP USER {#drop-user-statement} - -Удаляет пользователя. - -### Синтаксис {#drop-user-syntax} - -```sql -DROP USER [IF EXISTS] name [,...] [ON CLUSTER cluster_name] -``` - - -## DROP ROLE {#drop-role-statement} - -Удаляет роль. - -При удалении роль отзывается у всех объектов системы доступа, которым она присвоена. - -### Синтаксис {#drop-role-syntax} - -```sql -DROP ROLE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] -``` - -## DROP ROW POLICY {#drop-row-policy-statement} - -Удаляет политику доступа к строкам. - -При удалении политика отзывается у всех объектов системы доступа, которым она присвоена. - -### Синтаксис {#drop-row-policy-syntax} - -``` sql -DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] [ON CLUSTER cluster_name] -``` - - -## DROP QUOTA {#drop-quota-statement} - -Удаляет квоту. - -При удалении квота отзывается у всех объектов системы доступа, которым она присвоена. - -### Синтаксис {#drop-quota-syntax} - -``` sql -DROP QUOTA [IF EXISTS] name [,...] [ON CLUSTER cluster_name] -``` - - -## DROP SETTINGS PROFILE {#drop-settings-profile-statement} - -Удаляет профиль настроек. - -При удалении профиль отзывается у всех объектов системы доступа, которым он присвоен. - -### Синтаксис {#drop-settings-profile-syntax} - -``` sql -DROP [SETTINGS] PROFILE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] -``` - - -## EXISTS {#exists} - -``` sql -EXISTS [TEMPORARY] TABLE [db.]name [INTO OUTFILE filename] [FORMAT format] -``` - -Возвращает один столбец типа `UInt8`, содержащий одно значение - `0`, если таблицы или БД не существует и `1`, если таблица в указанной БД существует. - -## KILL QUERY {#kill-query-statement} - -``` sql -KILL QUERY [ON CLUSTER cluster] - WHERE - [SYNC|ASYNC|TEST] - [FORMAT format] -``` - -Пытается принудительно остановить исполняющиеся в данный момент запросы. -Запросы для принудительной остановки выбираются из таблицы system.processes с помощью условия, указанного в секции `WHERE` запроса `KILL`. - -Примеры - -``` sql --- Принудительно останавливает все запросы с указанным query_id: -KILL QUERY WHERE query_id='2-857d-4a57-9ee0-327da5d60a90' - --- Синхронно останавливает все запросы пользователя 'username': -KILL QUERY WHERE user='username' SYNC -``` - -Readonly-пользователи могут останавливать только свои запросы. - -По умолчанию используется асинхронный вариант запроса (`ASYNC`), который не дожидается подтверждения остановки запросов. - -Синхронный вариант (`SYNC`) ожидает остановки всех запросов и построчно выводит информацию о процессах по ходу их остановки. -Ответ содержит колонку `kill_status`, которая может принимать следующие значения: - -1. ‘finished’ - запрос был успешно остановлен; -2. ‘waiting’ - запросу отправлен сигнал завершения, ожидается его остановка; -3. остальные значения описывают причину невозможности остановки запроса. - -Тестовый вариант запроса (`TEST`) только проверяет права пользователя и выводит список запросов для остановки. - -## KILL MUTATION {#kill-mutation-statement} - -``` sql -KILL MUTATION [ON CLUSTER cluster] - WHERE - [TEST] - [FORMAT format] -``` - -Пытается остановить выполняющиеся в данные момент [мутации](alter.md#mutations). Мутации для остановки выбираются из таблицы [`system.mutations`](../../operations/system-tables/mutations.md#system_tables-mutations) с помощью условия, указанного в секции `WHERE` запроса `KILL`. - -Тестовый вариант запроса (`TEST`) только проверяет права пользователя и выводит список запросов для остановки. - -Примеры: - -``` sql --- Останавливает все мутации одной таблицы: -KILL MUTATION WHERE database = 'default' AND table = 'table' - --- Останавливает конкретную мутацию: -KILL MUTATION WHERE database = 'default' AND table = 'table' AND mutation_id = 'mutation_3.txt' -``` - -Запрос полезен в случаях, когда мутация не может выполниться до конца (например, если функция в запросе мутации бросает исключение на данных таблицы). - -Данные, уже изменённые мутацией, остаются в таблице (отката на старую версию данных не происходит). - -## OPTIMIZE {#misc_operations-optimize} - -``` sql -OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL] [DEDUPLICATE] -``` - -Запрос пытается запустить внеплановый мёрж кусков данных для таблиц семейства [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). Другие движки таблиц не поддерживаются. - -Если `OPTIMIZE` применяется к таблицам семейства [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md), ClickHouse создаёт задачу на мёрж и ожидает её исполнения на всех узлах (если активирована настройка `replication_alter_partitions_sync`). - -- Если `OPTIMIZE` не выполняет мёрж по любой причине, ClickHouse не оповещает об этом клиента. Чтобы включить оповещения, используйте настройку [optimize\_throw\_if\_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop). -- Если указать `PARTITION`, то оптимизация выполняется только для указанной партиции. [Как задавать имя партиции в запросах](alter.md#alter-how-to-specify-part-expr). -- Если указать `FINAL`, то оптимизация выполняется даже в том случае, если все данные уже лежат в одном куске. -- Если указать `DEDUPLICATE`, то произойдет схлопывание полностью одинаковых строк (сравниваются значения во всех колонках), имеет смысл только для движка MergeTree. - -!!! warning "Внимание" - Запрос `OPTIMIZE` не может устранить причину появления ошибки «Too many parts». - -## RENAME {#misc_operations-rename} - -Переименовывает одну или несколько таблиц. - -``` sql -RENAME TABLE [db11.]name11 TO [db12.]name12, [db21.]name21 TO [db22.]name22, ... [ON CLUSTER cluster] -``` - -Переименовывание таблицы является лёгкой операцией. Если вы указали после `TO` другую базу данных, то таблица будет перенесена в эту базу данных. При этом, директории с базами данных должны быть расположены в одной файловой системе (иначе возвращается ошибка). В случае переименования нескольких таблиц в одном запросе — это неатомарная операция, может выполнится частично, запросы в других сессиях могут получить ошибку `Table ... doesn't exist...`. - -## SET {#query-set} - -``` sql -SET param = value -``` - -Устанавливает значение `value` для [настройки](../../operations/settings/index.md) `param` в текущей сессии. [Конфигурационные параметры сервера](../../operations/server-configuration-parameters/settings.md) нельзя изменить подобным образом. - -Можно одним запросом установить все настройки из заданного профиля настроек. - -``` sql -SET profile = 'profile-name-from-the-settings-file' -``` - -Подробности смотрите в разделе [Настройки](../../operations/settings/settings.md). - -## SET ROLE {#set-role-statement} - -Активирует роли для текущего пользователя. - -### Синтаксис {#set-role-syntax} - -``` sql -SET ROLE {DEFAULT | NONE | role [,...] | ALL | ALL EXCEPT role [,...]} -``` - -## SET DEFAULT ROLE {#set-default-role-statement} - -Устанавливает роли по умолчанию для пользователя. - -Роли по умолчанию активируются автоматически при входе пользователя. Ролями по умолчанию могут быть установлены только ранее назначенные роли. Если роль не назначена пользователю, ClickHouse выбрасывает исключение. - - -### Синтаксис {#set-default-role-syntax} - -``` sql -SET DEFAULT ROLE {NONE | role [,...] | ALL | ALL EXCEPT role [,...]} TO {user|CURRENT_USER} [,...] -``` - - -### Примеры {#set-default-role-examples} - -Установить несколько ролей по умолчанию для пользователя: - -``` sql -SET DEFAULT ROLE role1, role2, ... TO user -``` - -Установить ролями по умолчанию все назначенные пользователю роли: - -``` sql -SET DEFAULT ROLE ALL TO user -``` - -Удалить роли по умолчанию для пользователя: - -``` sql -SET DEFAULT ROLE NONE TO user -``` - -Установить ролями по умолчанию все назначенные пользователю роли за исключением указанных: - -```sql -SET DEFAULT ROLE ALL EXCEPT role1, role2 TO user -``` - -## TRUNCATE {#truncate-statement} - -``` sql -TRUNCATE TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] -``` - -Удаляет все данные из таблицы. Если условие `IF EXISTS` не указано, запрос вернет ошибку, если таблицы не существует. - -Запрос `TRUNCATE` не поддерживается для следующих движков: [View](../../engines/table-engines/special/view.md), [File](../../engines/table-engines/special/file.md), [URL](../../engines/table-engines/special/url.md) и [Null](../../engines/table-engines/special/null.md). - -## USE {#use} - -``` sql -USE db -``` - -Позволяет установить текущую базу данных для сессии. -Текущая база данных используется для поиска таблиц, если база данных не указана в запросе явно через точку перед именем таблицы. -При использовании HTTP протокола запрос не может быть выполнен, так как понятия сессии не существует. - -[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/misc/) +- [ATTACH](../../sql-reference/statements/attach.md) +- [CHECK TABLE](../../sql-reference/statements/check-table.md) +- [DESCRIBE TABLE](../../sql-reference/statements/describe-table.md) +- [DETACH](../../sql-reference/statements/detach.md) +- [DROP](../../sql-reference/statements/drop.md) +- [EXISTS](../../sql-reference/statements/exists.md) +- [KILL](../../sql-reference/statements/kill.md) +- [OPTIMIZE](../../sql-reference/statements/optimize.md) +- [RENAME](../../sql-reference/statements/rename.md) +- [SET](../../sql-reference/statements/set.md) +- [SET ROLE](../../sql-reference/statements/set-role.md) +- [TRUNCATE](../../sql-reference/statements/truncate.md) +- [USE](../../sql-reference/statements/use.md) + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/misc/) \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/optimize.md b/docs/ru/sql-reference/statements/optimize.md new file mode 100644 index 00000000000..c150f02c3d6 --- /dev/null +++ b/docs/ru/sql-reference/statements/optimize.md @@ -0,0 +1,25 @@ +--- +toc_priority: 49 +toc_title: OPTIMIZE +--- + +# OPTIMIZE {#misc_operations-optimize} + +``` sql +OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL] [DEDUPLICATE] +``` + +Запрос пытается запустить внеплановый мёрж кусков данных для таблиц семейства [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). Другие движки таблиц не поддерживаются. + +Если `OPTIMIZE` применяется к таблицам семейства [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md), ClickHouse создаёт задачу на мёрж и ожидает её исполнения на всех узлах (если активирована настройка `replication_alter_partitions_sync`). + +- Если `OPTIMIZE` не выполняет мёрж по любой причине, ClickHouse не оповещает об этом клиента. Чтобы включить оповещения, используйте настройку [optimize\_throw\_if\_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop). +- Если указать `PARTITION`, то оптимизация выполняется только для указанной партиции. [Как задавать имя партиции в запросах](alter.md#alter-how-to-specify-part-expr). +- Если указать `FINAL`, то оптимизация выполняется даже в том случае, если все данные уже лежат в одном куске. +- Если указать `DEDUPLICATE`, то произойдет схлопывание полностью одинаковых строк (сравниваются значения во всех колонках), имеет смысл только для движка MergeTree. + +!!! warning "Внимание" + Запрос `OPTIMIZE` не может устранить причину появления ошибки «Too many parts». + + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/optimize/) \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/rename.md b/docs/ru/sql-reference/statements/rename.md new file mode 100644 index 00000000000..9f6069d8a94 --- /dev/null +++ b/docs/ru/sql-reference/statements/rename.md @@ -0,0 +1,17 @@ +--- +toc_priority: 50 +toc_title: RENAME +--- + +# RENAME {#misc_operations-rename} + +Переименовывает одну или несколько таблиц. + +``` sql +RENAME TABLE [db11.]name11 TO [db12.]name12, [db21.]name21 TO [db22.]name22, ... [ON CLUSTER cluster] +``` + +Переименовывание таблицы является лёгкой операцией. Если вы указали после `TO` другую базу данных, то таблица будет перенесена в эту базу данных. При этом, директории с базами данных должны быть расположены в одной файловой системе (иначе возвращается ошибка). В случае переименования нескольких таблиц в одном запросе — это неатомарная операция, может выполнится частично, запросы в других сессиях могут получить ошибку `Table ... doesn't exist...`. + + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/rename/) diff --git a/docs/ru/sql-reference/statements/set-role.md b/docs/ru/sql-reference/statements/set-role.md new file mode 100644 index 00000000000..5d840fcddb7 --- /dev/null +++ b/docs/ru/sql-reference/statements/set-role.md @@ -0,0 +1,57 @@ +--- +toc_priority: 52 +toc_title: SET ROLE +--- + +# SET ROLE {#set-role-statement} + +Активирует роли для текущего пользователя. + +### Синтаксис {#set-role-syntax} + +``` sql +SET ROLE {DEFAULT | NONE | role [,...] | ALL | ALL EXCEPT role [,...]} +``` + +## SET DEFAULT ROLE {#set-default-role-statement} + +Устанавливает роли по умолчанию для пользователя. + +Роли по умолчанию активируются автоматически при входе пользователя. Ролями по умолчанию могут быть установлены только ранее назначенные роли. Если роль не назначена пользователю, ClickHouse выбрасывает исключение. + + +### Синтаксис {#set-default-role-syntax} + +``` sql +SET DEFAULT ROLE {NONE | role [,...] | ALL | ALL EXCEPT role [,...]} TO {user|CURRENT_USER} [,...] +``` + + +### Примеры {#set-default-role-examples} + +Установить несколько ролей по умолчанию для пользователя: + +``` sql +SET DEFAULT ROLE role1, role2, ... TO user +``` + +Установить ролями по умолчанию все назначенные пользователю роли: + +``` sql +SET DEFAULT ROLE ALL TO user +``` + +Удалить роли по умолчанию для пользователя: + +``` sql +SET DEFAULT ROLE NONE TO user +``` + +Установить ролями по умолчанию все назначенные пользователю роли за исключением указанных: + +```sql +SET DEFAULT ROLE ALL EXCEPT role1, role2 TO user +``` + + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/set-role/) \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/set.md b/docs/ru/sql-reference/statements/set.md new file mode 100644 index 00000000000..0e12e2ee7bc --- /dev/null +++ b/docs/ru/sql-reference/statements/set.md @@ -0,0 +1,22 @@ +--- +toc_priority: 51 +toc_title: SET +--- + +# SET {#query-set} + +``` sql +SET param = value +``` + +Устанавливает значение `value` для [настройки](../../operations/settings/index.md) `param` в текущей сессии. [Конфигурационные параметры сервера](../../operations/server-configuration-parameters/settings.md) нельзя изменить подобным образом. + +Можно одним запросом установить все настройки из заданного профиля настроек. + +``` sql +SET profile = 'profile-name-from-the-settings-file' +``` + +Подробности смотрите в разделе [Настройки](../../operations/settings/settings.md). + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/set/) \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/truncate.md b/docs/ru/sql-reference/statements/truncate.md new file mode 100644 index 00000000000..f8806496e48 --- /dev/null +++ b/docs/ru/sql-reference/statements/truncate.md @@ -0,0 +1,17 @@ +--- +toc_priority: 53 +toc_title: TRUNCATE +--- + +# TRUNCATE {#truncate-statement} + +``` sql +TRUNCATE TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] +``` + +Удаляет все данные из таблицы. Если условие `IF EXISTS` не указано, запрос вернет ошибку, если таблицы не существует. + +Запрос `TRUNCATE` не поддерживается для следующих движков: [View](../../engines/table-engines/special/view.md), [File](../../engines/table-engines/special/file.md), [URL](../../engines/table-engines/special/url.md) и [Null](../../engines/table-engines/special/null.md). + + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/truncate/) \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/use.md b/docs/ru/sql-reference/statements/use.md new file mode 100644 index 00000000000..84b0f5ed13f --- /dev/null +++ b/docs/ru/sql-reference/statements/use.md @@ -0,0 +1,16 @@ +--- +toc_priority: 54 +toc_title: USE +--- + +# USE {#use} + +``` sql +USE db +``` + +Позволяет установить текущую базу данных для сессии. +Текущая база данных используется для поиска таблиц, если база данных не указана в запросе явно через точку перед именем таблицы. +При использовании HTTP протокола запрос не может быть выполнен, так как понятия сессии не существует. + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/use/) \ No newline at end of file From f5b346d35075ebac1ec9c53cbe96942b1bf41178 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 10 Aug 2020 16:16:23 +0300 Subject: [PATCH 227/374] Update CHANGELOG.md --- CHANGELOG.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 16e3d3a5935..f8f91b63128 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,14 +4,14 @@ #### New Feature +* Added an initial implementation of `EXPLAIN` query. Syntax: `EXPLAIN SELECT ...`. This fixes [#1118](https://github.com/ClickHouse/ClickHouse/issues/1118). [#11873](https://github.com/ClickHouse/ClickHouse/pull/11873) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Added storage `RabbitMQ`. [#11069](https://github.com/ClickHouse/ClickHouse/pull/11069) ([Kseniia Sumarokova](https://github.com/kssenii)). * Implemented PostgreSQL-like `ILIKE` operator for [#11710](https://github.com/ClickHouse/ClickHouse/issues/11710). [#12125](https://github.com/ClickHouse/ClickHouse/pull/12125) ([Mike](https://github.com/myrrc)). * Supported RIGHT and FULL JOIN with `set join_algorithm=partial_merge`. Only ALL strictness is allowed (ANY, SEMI, ANTI, ASOF are not). [#12118](https://github.com/ClickHouse/ClickHouse/pull/12118) ([Artem Zuikov](https://github.com/4ertus2)). * Added a function `initializedAggregation` to initialize an aggregation based on a single value. [#12109](https://github.com/ClickHouse/ClickHouse/pull/12109) ([Guillaume Tassery](https://github.com/YiuRULE)). * Supported `ALTER TABLE ... [ADD|MODIFY] COLUMN ... FIRST` [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#12073](https://github.com/ClickHouse/ClickHouse/pull/12073) ([Winter Zhang](https://github.com/zhang2014)). * Added function `parseDateTimeBestEffortUS`. [#12028](https://github.com/ClickHouse/ClickHouse/pull/12028) ([flynn](https://github.com/ucasFL)). -* Added an initial implementation of `EXPLAIN` query. Syntax: `EXPLAIN SELECT ...`. This fixes [#1118](https://github.com/ClickHouse/ClickHouse/issues/1118). [#11873](https://github.com/ClickHouse/ClickHouse/pull/11873) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Added `ORCBlockOutputFormat`. [#11662](https://github.com/ClickHouse/ClickHouse/pull/11662) ([Kruglov Pavel](https://github.com/Avogar)). -* Added storage `RabbitMQ`. [#11069](https://github.com/ClickHouse/ClickHouse/pull/11069) ([Kseniia Sumarokova](https://github.com/kssenii)). #### Bug Fix @@ -109,9 +109,9 @@ #### Performance Improvement +* Improved performace of 'ORDER BY' and 'GROUP BY' by prefix of sorting key (enabled with `optimize_aggregation_in_order` setting, disabled by default). [#11696](https://github.com/ClickHouse/ClickHouse/pull/11696) ([Anton Popov](https://github.com/CurtizJ)). * Removed injective functions inside `uniq*()` if `set optimize_injective_functions_inside_uniq=1`. [#12337](https://github.com/ClickHouse/ClickHouse/pull/12337) ([Ruslan Kamalov](https://github.com/kamalov-ruslan)). * Index not used for IN operator with literals", performance regression introduced around v19.3. This fixes "[#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). -* Improved performace of 'ORDER BY' and 'GROUP BY' by prefix of sorting key. [#11696](https://github.com/ClickHouse/ClickHouse/pull/11696) ([Anton Popov](https://github.com/CurtizJ)). #### Experimental Feature * Added new in-memory format of parts in `MergeTree`-family tables, which stores data in memory. Parts are written on disk at first merge. Part will be created in in-memory format if its size in rows or bytes is below thresholds `min_rows_for_compact_part` and `min_bytes_for_compact_part`. Also optional support of Write-Ahead-Log is available, which is enabled by default and is controlled by setting `in_memory_parts_enable_wal`. [#10697](https://github.com/ClickHouse/ClickHouse/pull/10697) ([Anton Popov](https://github.com/CurtizJ)). From 5833f090c64dae0b9abb595deaff65ef51e4def3 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 10 Aug 2020 16:17:11 +0300 Subject: [PATCH 228/374] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f8f91b63128..bd0ebb5d6b2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -11,7 +11,7 @@ * Added a function `initializedAggregation` to initialize an aggregation based on a single value. [#12109](https://github.com/ClickHouse/ClickHouse/pull/12109) ([Guillaume Tassery](https://github.com/YiuRULE)). * Supported `ALTER TABLE ... [ADD|MODIFY] COLUMN ... FIRST` [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#12073](https://github.com/ClickHouse/ClickHouse/pull/12073) ([Winter Zhang](https://github.com/zhang2014)). * Added function `parseDateTimeBestEffortUS`. [#12028](https://github.com/ClickHouse/ClickHouse/pull/12028) ([flynn](https://github.com/ucasFL)). -* Added `ORCBlockOutputFormat`. [#11662](https://github.com/ClickHouse/ClickHouse/pull/11662) ([Kruglov Pavel](https://github.com/Avogar)). +* Support format `ORC` for output (was supported only for input). [#11662](https://github.com/ClickHouse/ClickHouse/pull/11662) ([Kruglov Pavel](https://github.com/Avogar)). #### Bug Fix From b6e5f69802f8ce6299c703e40b0b448b7d8ceab6 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 10 Aug 2020 16:17:40 +0300 Subject: [PATCH 229/374] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index bd0ebb5d6b2..77e88c9d3a8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,7 +7,7 @@ * Added an initial implementation of `EXPLAIN` query. Syntax: `EXPLAIN SELECT ...`. This fixes [#1118](https://github.com/ClickHouse/ClickHouse/issues/1118). [#11873](https://github.com/ClickHouse/ClickHouse/pull/11873) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Added storage `RabbitMQ`. [#11069](https://github.com/ClickHouse/ClickHouse/pull/11069) ([Kseniia Sumarokova](https://github.com/kssenii)). * Implemented PostgreSQL-like `ILIKE` operator for [#11710](https://github.com/ClickHouse/ClickHouse/issues/11710). [#12125](https://github.com/ClickHouse/ClickHouse/pull/12125) ([Mike](https://github.com/myrrc)). -* Supported RIGHT and FULL JOIN with `set join_algorithm=partial_merge`. Only ALL strictness is allowed (ANY, SEMI, ANTI, ASOF are not). [#12118](https://github.com/ClickHouse/ClickHouse/pull/12118) ([Artem Zuikov](https://github.com/4ertus2)). +* Supported RIGHT and FULL JOIN with `SET join_algorithm = 'partial_merge'`. Only ALL strictness is allowed (ANY, SEMI, ANTI, ASOF are not). [#12118](https://github.com/ClickHouse/ClickHouse/pull/12118) ([Artem Zuikov](https://github.com/4ertus2)). * Added a function `initializedAggregation` to initialize an aggregation based on a single value. [#12109](https://github.com/ClickHouse/ClickHouse/pull/12109) ([Guillaume Tassery](https://github.com/YiuRULE)). * Supported `ALTER TABLE ... [ADD|MODIFY] COLUMN ... FIRST` [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#12073](https://github.com/ClickHouse/ClickHouse/pull/12073) ([Winter Zhang](https://github.com/zhang2014)). * Added function `parseDateTimeBestEffortUS`. [#12028](https://github.com/ClickHouse/ClickHouse/pull/12028) ([flynn](https://github.com/ucasFL)). From 65da9739440e888e5d32ccd2b34182fb47c27c16 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 10 Aug 2020 16:20:29 +0300 Subject: [PATCH 230/374] Update CHANGELOG.md --- CHANGELOG.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 77e88c9d3a8..5850c70043d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -94,7 +94,7 @@ #### Improvement -* Allowed to set `JOIN` kind and type in more standart way: `LEFT SEMI JOIN` instead of `SEMI LEFT JOIN`. For now both are correct. [#12520](https://github.com/ClickHouse/ClickHouse/pull/12520) ([Artem Zuikov](https://github.com/4ertus2)). +* Allowed to set `JOIN` kind and type in more standard way: `LEFT SEMI JOIN` instead of `SEMI LEFT JOIN`. For now both are correct. [#12520](https://github.com/ClickHouse/ClickHouse/pull/12520) ([Artem Zuikov](https://github.com/4ertus2)). * lifetime_rows/lifetime_bytes for Buffer engine. [#12421](https://github.com/ClickHouse/ClickHouse/pull/12421) ([Azat Khuzhin](https://github.com/azat)). * Write the detail exception message to the client instead of 'MySQL server has gone away'. [#12383](https://github.com/ClickHouse/ClickHouse/pull/12383) ([BohuTANG](https://github.com/BohuTANG)). * Allows to change a charset which is used for printing grids borders. Available charsets are following: UTF-8, ASCII. Setting `output_format_pretty_grid_charset` enables this feature. [#12372](https://github.com/ClickHouse/ClickHouse/pull/12372) ([Sabyanin Maxim](https://github.com/s-mx)). @@ -103,7 +103,6 @@ * Added support for `%g` (two digit ISO year) and `%G` (four digit ISO year) substitutions in `formatDateTime` function. [#12136](https://github.com/ClickHouse/ClickHouse/pull/12136) ([vivarum](https://github.com/vivarum)). * Added 'type' column in system.disks. [#12115](https://github.com/ClickHouse/ClickHouse/pull/12115) ([ianton-ru](https://github.com/ianton-ru)). * Improved `REVOKE` command: now it requires grant/admin option for only access which will be revoked. For example, to execute `REVOKE ALL ON *.* FROM user1` now it doesn't require to have full access rights granted with grant option. Added command `REVOKE ALL FROM user1` - it revokes all granted roles from `user1`. [#12083](https://github.com/ClickHouse/ClickHouse/pull/12083) ([Vitaly Baranov](https://github.com/vitlibar)). -* Implemented single part uploads for DiskS3. [#12026](https://github.com/ClickHouse/ClickHouse/pull/12026) ([Vladimir Chebotarev](https://github.com/excitoon)). * Added replica priority for load_balancing (for manual prioritization of the load balancing). [#11995](https://github.com/ClickHouse/ClickHouse/pull/11995) ([Azat Khuzhin](https://github.com/azat)). * Switched paths in S3 metadata to relative which allows to handle S3 blobs more easily. [#11892](https://github.com/ClickHouse/ClickHouse/pull/11892) ([Vladimir Chebotarev](https://github.com/excitoon)). @@ -112,6 +111,7 @@ * Improved performace of 'ORDER BY' and 'GROUP BY' by prefix of sorting key (enabled with `optimize_aggregation_in_order` setting, disabled by default). [#11696](https://github.com/ClickHouse/ClickHouse/pull/11696) ([Anton Popov](https://github.com/CurtizJ)). * Removed injective functions inside `uniq*()` if `set optimize_injective_functions_inside_uniq=1`. [#12337](https://github.com/ClickHouse/ClickHouse/pull/12337) ([Ruslan Kamalov](https://github.com/kamalov-ruslan)). * Index not used for IN operator with literals", performance regression introduced around v19.3. This fixes "[#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). +* Implemented single part uploads for DiskS3 (experimental feature). [#12026](https://github.com/ClickHouse/ClickHouse/pull/12026) ([Vladimir Chebotarev](https://github.com/excitoon)). #### Experimental Feature * Added new in-memory format of parts in `MergeTree`-family tables, which stores data in memory. Parts are written on disk at first merge. Part will be created in in-memory format if its size in rows or bytes is below thresholds `min_rows_for_compact_part` and `min_bytes_for_compact_part`. Also optional support of Write-Ahead-Log is available, which is enabled by default and is controlled by setting `in_memory_parts_enable_wal`. [#10697](https://github.com/ClickHouse/ClickHouse/pull/10697) ([Anton Popov](https://github.com/CurtizJ)). From 6164e7373d507ef7bee9ecf403017af20f90d803 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Mon, 10 Aug 2020 16:33:39 +0300 Subject: [PATCH 231/374] DOCSUP-791: Docs for optimize_read_in_order setting (#12611) * Docs in English for optimize_read_in_order setting. * Link misspelling fixed. * Usage section moved to the ORDER BY topic. * Link fixed. * Update docs/en/sql-reference/statements/select/order-by.md Co-authored-by: BayoNet * Update docs/en/sql-reference/statements/select/order-by.md Co-authored-by: BayoNet * Apply suggestions from code review Co-authored-by: BayoNet * Apply suggestions from code review Co-authored-by: BayoNet * Update docs/en/sql-reference/statements/select/order-by.md Co-authored-by: Ivan Blinkov * Small fix in English and translation into Russian. Co-authored-by: Olga Revyakina Co-authored-by: BayoNet Co-authored-by: Ivan Blinkov --- docs/en/operations/settings/settings.md | 14 ++++++++++++++ .../statements/select/order-by.md | 19 +++++++++++++++++++ docs/ru/operations/settings/settings.md | 16 ++++++++++++++++ .../statements/select/order-by.md | 19 +++++++++++++++++++ 4 files changed, 68 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index aaca694dc06..27d5a2e6f79 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1722,6 +1722,20 @@ SELECT * FROM a; | 1 | +---+ ``` +## optimize_read_in_order {#optimize_read_in_order} + +Enables [ORDER BY](../../sql-reference/statements/select/order-by.md#optimize_read_in_order) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries for reading data from [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) tables. + +Possible values: + +- 0 — `ORDER BY` optimization is disabled. +- 1 — `ORDER BY` optimization is enabled. + +Default value: `1`. + +**See Also** + +- [ORDER BY Clause](../../sql-reference/statements/select/order-by.md#optimize_read_in_order) ## mutations_sync {#mutations_sync} diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 58f3aa7597f..a4e5e3655c6 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -70,6 +70,25 @@ Running a query may use more memory than `max_bytes_before_external_sort`. For t External sorting works much less effectively than sorting in RAM. +## Optimization of Data Reading {#optimize_read_in_order} + + If `ORDER BY` expression has a prefix that coincides with the table sorting key, you can optimize the query by using the [optimize_read_in_order](../../../operations/settings/settings.md#optimize_read_in_order) setting. + + When the `optimize_read_in_order` setting is enabled, the Clickhouse server uses the table index and reads the data in order of the `ORDER BY` key. This allows to avoid reading all data in case of specified [LIMIT](../../../sql-reference/statements/select/limit.md). So queries on big data with small limit are processed faster. + +Optimization works with both `ASC` and `DESC` and doesn't work together with [GROUP BY](../../../sql-reference/statements/select/group-by.md) clause and [FINAL](../../../sql-reference/statements/select/from.md#select-from-final) modifier. + +When the `optimize_read_in_order` setting is disabled, the Clickhouse server does not use the table index while processing `SELECT` queries. + +Consider disabling `optimize_read_in_order` manually, when running queries that have `ORDER BY` clause, large `LIMIT` and [WHERE](../../../sql-reference/statements/select/where.md) condition that requires to read huge amount of records before queried data is found. + +Optimization is supported in the following table engines: + +- [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md) +- [Merge](../../../engines/table-engines/special/merge.md), [Buffer](../../../engines/table-engines/special/buffer.md), and [MaterializedView](../../../engines/table-engines/special/materializedview.md) table engines over `MergeTree`-engine tables + +In `MaterializedView`-engine tables the optimization works with views like `SELECT ... FROM merge_tree_table ORDER BY pk`. But it is not supported in the queries like `SELECT ... FROM view ORDER BY pk` if the view query doesn't have the `ORDER BY` clause. + ## ORDER BY Expr WITH FILL Modifier {#orderby-with-fill} This modifier also can be combined with [LIMIT … WITH TIES modifier](../../../sql-reference/statements/select/limit.md#limit-with-ties). diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index bbedeb95d49..589b88597c8 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1480,6 +1480,21 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; - [min_insert_block_size_bytes](#min-insert-block-size-bytes) +## optimize_read_in_order {#optimize_read_in_order} + +Включает или отключает оптимизацию в запросах [SELECT](../../sql-reference/statements/select/index.md) с секцией [ORDER BY](../../sql-reference/statements/select/order-by.md#optimize_read_in_order) при работе с таблицами семейства [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). + +Возможные значения: + +- 0 — оптимизация отключена. +- 1 — оптимизация включена. + +Значение по умолчанию: `1`. + +**См. также** + +- [Оптимизация чтения данных](../../sql-reference/statements/select/order-by.md#optimize_read_in_order) в секции `ORDER BY` + ## mutations_sync {#mutations_sync} Позволяет выполнять запросы `ALTER TABLE ... UPDATE|DELETE` ([мутации](../../sql-reference/statements/alter.md#mutations)) синхронно. @@ -1497,4 +1512,5 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; - [Синхронность запросов ALTER](../../sql-reference/statements/alter.md#synchronicity-of-alter-queries) - [Мутации](../../sql-reference/statements/alter.md#mutations) + [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) diff --git a/docs/ru/sql-reference/statements/select/order-by.md b/docs/ru/sql-reference/statements/select/order-by.md index 39126755d33..0476c5da5af 100644 --- a/docs/ru/sql-reference/statements/select/order-by.md +++ b/docs/ru/sql-reference/statements/select/order-by.md @@ -70,6 +70,25 @@ toc_title: ORDER BY Внешняя сортировка работает существенно менее эффективно, чем сортировка в оперативке. +## Оптимизация чтения данных {#optimize_read_in_order} + + Если в списке выражений в секции `ORDER BY` первыми указаны те поля, по которым проиндексирована таблица, по которой строится выборка, такой запрос можно оптимизировать — для этого используйте настройку [optimize_read_in_order](../../../operations/settings/settings.md#optimize_read_in_order). + + Когда настройка `optimize_read_in_order` включена, при выполнении запроса сервер использует табличные индексы и считывает данные в том порядке, который задан списком выражений `ORDER BY`. Поэтому если в запросе установлен [LIMIT](../../../sql-reference/statements/select/limit.md), сервер не станет считывать лишние данные. Таким образом, запросы к большим таблицам, но имеющие ограничения по числу записей, выполняются быстрее. + +Оптимизация работает при любом порядке сортировки `ASC` или `DESC`, но не работает при использовании группировки [GROUP BY](../../../sql-reference/statements/select/group-by.md) и модификатора [FINAL](../../../sql-reference/statements/select/from.md#select-from-final). + +Когда настройка `optimize_read_in_order` отключена, при выполнении запросов `SELECT` табличные индексы не используются. + +Для запросов с сортировкой `ORDER BY`, большим значением `LIMIT` и условиями отбора [WHERE](../../../sql-reference/statements/select/where.md), требующими чтения больших объемов данных, рекомендуется отключать `optimize_read_in_order` вручную. + +Оптимизация чтения данных поддерживается в следующих движках: + +- [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md) +- [Merge](../../../engines/table-engines/special/merge.md), [Buffer](../../../engines/table-engines/special/buffer.md) и [MaterializedView](../../../engines/table-engines/special/materializedview.md), работающими с таблицами `MergeTree` + +В движке `MaterializedView` оптимизация поддерживается при работе с сохраненными запросами (представлениями) вида `SELECT ... FROM merge_tree_table ORDER BY pk`. Но оптимизация не поддерживается для запросов вида `SELECT ... FROM view ORDER BY pk`, если в сохраненном запросе нет секции `ORDER BY`. + ## Модификатор ORDER BY expr WITH FILL {#orderby-with-fill} Этот модификатор также может быть скобинирован с модификатором [LIMIT ... WITH TIES](../../../sql-reference/statements/select/limit.md#limit-with-ties) From 4c61a29a630d7195ef11f7dcc270aff28396f394 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 10 Aug 2020 16:44:04 +0300 Subject: [PATCH 232/374] Add noop runner parameter to prepare actual changes --- tests/integration/runner | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/integration/runner b/tests/integration/runner index 6369ebeea3e..71993e47ff7 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -124,6 +124,10 @@ if __name__ == "__main__": default="latest", help="Version of docker image which runner will use to run tests") + parser.add_argument( + "--docker-compose-images-tags", + action="append", + help="Set non-default tags for images used in docker compose recipes(yandex/my_container:my_tag)") parser.add_argument('pytest_args', nargs='*', help="args for pytest command") From 2311cda3349e0638901b57e89ff050344b6ab03b Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 10 Aug 2020 22:32:05 +0800 Subject: [PATCH 233/374] ISSUES-4006 convert _sign & _version to materialized column --- .../AddingVersionsBlockOutputStream.cpp | 57 -------------- .../AddingVersionsBlockOutputStream.h | 36 --------- src/Databases/MySQL/MaterializeMetadata.cpp | 2 +- src/Databases/MySQL/MaterializeMetadata.h | 2 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 77 +++++++++++++------ .../MySQL/InterpretersMySQLDDLQuery.cpp | 16 +++- src/Storages/StorageMaterializeMySQL.cpp | 23 +----- src/Storages/StorageMaterializeMySQL.h | 1 - 8 files changed, 73 insertions(+), 141 deletions(-) delete mode 100644 src/DataStreams/AddingVersionsBlockOutputStream.cpp delete mode 100644 src/DataStreams/AddingVersionsBlockOutputStream.h diff --git a/src/DataStreams/AddingVersionsBlockOutputStream.cpp b/src/DataStreams/AddingVersionsBlockOutputStream.cpp deleted file mode 100644 index b94c858b38e..00000000000 --- a/src/DataStreams/AddingVersionsBlockOutputStream.cpp +++ /dev/null @@ -1,57 +0,0 @@ -#include - -#include -#include - -namespace DB -{ - -void AddingVersionsBlockOutputStream::writePrefix() -{ - output->writePrefix(); -} - -void AddingVersionsBlockOutputStream::writeSuffix() -{ - output->writeSuffix(); -} - -void AddingVersionsBlockOutputStream::flush() -{ - output->flush(); -} - -void AddingVersionsBlockOutputStream::write(const Block & block) -{ - Block res; - size_t rows = block.rows(); - - for (size_t index = 0; index < block.columns(); ++index) - res.insert(block.getByPosition(index)); - - DataTypePtr sign_type = std::make_shared(); - DataTypePtr version_type = std::make_shared(); - - ColumnPtr sign_column = sign_type->createColumnConst(rows, Field(Int8(1)))->convertToFullColumnIfConst(); - ColumnPtr version_column = version_type->createColumnConst(rows, Field(UInt64(++version)))->convertToFullColumnIfConst(); - - Block header = output->getHeader(); - res.insert({sign_column, sign_type, header.getByPosition(header.columns() - 2).name}); - res.insert({version_column, version_type, header.getByPosition(header.columns() - 1).name}); - output->write(res); - - written_rows += block.rows(); - written_bytes += block.bytes(); -} -Block AddingVersionsBlockOutputStream::getHeader() const -{ - Block res; - Block header = output->getHeader(); - - for (size_t index = 0; index < header.columns() - 2; ++index) - res.insert(header.getByPosition(index)); - - return res; -} - -} diff --git a/src/DataStreams/AddingVersionsBlockOutputStream.h b/src/DataStreams/AddingVersionsBlockOutputStream.h deleted file mode 100644 index 4edc0ac13c7..00000000000 --- a/src/DataStreams/AddingVersionsBlockOutputStream.h +++ /dev/null @@ -1,36 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class AddingVersionsBlockOutputStream : public IBlockOutputStream -{ -public: - AddingVersionsBlockOutputStream(size_t & version_, const BlockOutputStreamPtr & output_) - : version(version_), output(output_) - { - } - - Block getHeader() const override; - - void write(const Block & block) override; - - void flush() override; - - void writePrefix() override; - void writeSuffix() override; - -private: - size_t & version; - BlockOutputStreamPtr output; - - std::atomic written_rows{0}, written_bytes{0}; - -public: - size_t getWrittenRows() { return written_rows; } - size_t getWrittenBytes() { return written_bytes; } -}; - -} diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 80484c130d3..3818761cc11 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -80,7 +80,7 @@ void MaterializeMetadata::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & c if (!master_status || master_status.rows() != 1) throw Exception("Unable to get master status from MySQL.", ErrorCodes::LOGICAL_ERROR); - version = 0; + version = 1; binlog_file = (*master_status.getByPosition(0).column)[0].safeGet(); binlog_position = (*master_status.getByPosition(1).column)[0].safeGet(); binlog_do_db = (*master_status.getByPosition(2).column)[0].safeGet(); diff --git a/src/Databases/MySQL/MaterializeMetadata.h b/src/Databases/MySQL/MaterializeMetadata.h index 045dab5b84c..3fcae3dcf1c 100644 --- a/src/Databases/MySQL/MaterializeMetadata.h +++ b/src/Databases/MySQL/MaterializeMetadata.h @@ -32,7 +32,7 @@ struct MaterializeMetadata String binlog_ignore_db; String executed_gtid_set; - size_t version = 0; + size_t version = 1; std::unordered_map need_dumping_tables; void fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & connection); diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index b53d84189ab..ebe93f527d3 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -9,7 +9,7 @@ # include # include # include -# include +# include # include # include # include @@ -37,20 +37,28 @@ namespace ErrorCodes static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; -static BlockIO tryToExecuteQuery(const String & query_to_execute, const Context & context_, const String & database, const String & comment) +static Context createQueryContext(const Context & global_context) +{ + Settings new_query_settings = global_context.getSettings(); + new_query_settings.insert_allow_materialized_columns = true; + + Context query_context(global_context); + query_context.setSettings(new_query_settings); + CurrentThread::QueryScope query_scope(query_context); + + query_context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + query_context.setCurrentQueryId(""); // generate random query_id + return query_context; +} + +static BlockIO tryToExecuteQuery(const String & query_to_execute, Context & query_context, const String & database, const String & comment) { try { - Context context(context_); - CurrentThread::QueryScope query_scope(context); - if (!database.empty()) - context.setCurrentDatabase(database); + query_context.setCurrentDatabase(database); - context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - context.setCurrentQueryId(""); // generate random query_id - - return executeQuery("/*" + comment + "*/ " + query_to_execute, context, true); + return executeQuery("/*" + comment + "*/ " + query_to_execute, query_context, true); } catch (...) { @@ -216,16 +224,35 @@ static inline void cleanOutdatedTables(const String & database_name, const Conte for (auto iterator = clean_database->getTablesIterator(context); iterator->isValid(); iterator->next()) { + Context query_context = createQueryContext(context); String comment = "Materialize MySQL step 1: execute MySQL DDL for dump data"; String table_name = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(iterator->name()); - tryToExecuteQuery(" DROP TABLE " + table_name, context, database_name, comment); + tryToExecuteQuery(" DROP TABLE " + table_name, query_context, database_name, comment); } } -static inline BlockOutputStreamPtr getTableOutput(const String & database_name, const String & table_name, const Context & context) +static inline BlockOutputStreamPtr getTableOutput(const String & database_name, const String & table_name, Context & query_context, bool insert_materialized = false) { + const StoragePtr & storage = DatabaseCatalog::instance().getTable(StorageID(database_name, table_name), query_context); + + std::stringstream insert_columns_str; + const StorageInMemoryMetadata & storage_metadata = storage->getInMemoryMetadata(); + const ColumnsDescription & storage_columns = storage_metadata.getColumns(); + const NamesAndTypesList & insert_columns_names = insert_materialized ? storage_columns.getAllPhysical() : storage_columns.getOrdinary(); + + + for (auto iterator = insert_columns_names.begin(); iterator != insert_columns_names.end(); ++iterator) + { + if (iterator != insert_columns_names.begin()) + insert_columns_str << ", "; + + insert_columns_str << iterator->name; + } + + String comment = "Materialize MySQL step 1: execute dump data"; - BlockIO res = tryToExecuteQuery("INSERT INTO " + backQuoteIfNeed(table_name) + " VALUES", context, database_name, comment); + BlockIO res = tryToExecuteQuery("INSERT INTO " + backQuoteIfNeed(table_name) + "(" + insert_columns_str.str() + ")" + " VALUES", + query_context, database_name, comment); if (!res.out) throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR); @@ -242,21 +269,23 @@ static inline void dumpDataForTables( for (; iterator != master_info.need_dumping_tables.end() && !is_cancelled(); ++iterator) { const auto & table_name = iterator->first; + Context query_context = createQueryContext(context); String comment = "Materialize MySQL step 1: execute MySQL DDL for dump data"; - tryToExecuteQuery(query_prefix + " " + iterator->second, context, database_name, comment); /// create table. + tryToExecuteQuery(query_prefix + " " + iterator->second, query_context, database_name, comment); /// create table. - auto out = std::make_shared(master_info.version, getTableOutput(database_name, table_name, context)); + auto out = std::make_shared(getTableOutput(database_name, table_name, query_context)); MySQLBlockInputStream input( connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name), out->getHeader(), DEFAULT_BLOCK_SIZE); Stopwatch watch; copyData(input, *out, is_cancelled); + const Progress & progress = out->getProgress(); LOG_INFO(&Poco::Logger::get("MaterializeMySQLSyncThread(" + database_name + ")"), - "Materialize MySQL step 1: dump {}, {} rows, {} in {} sec., {} rows/sec., {}/sec.", - table_name, out->getWrittenRows(), ReadableSize(out->getWrittenBytes()), watch.elapsedSeconds(), - static_cast(out->getWrittenRows() / watch.elapsedSeconds()), - ReadableSize(out->getWrittenRows() / watch.elapsedSeconds())); + "Materialize MySQL step 1: dump {}, {} rows, {} in {} sec., {} rows/sec., {}/sec." + , table_name, formatReadableQuantity(progress.written_rows), formatReadableSizeWithBinarySuffix(progress.written_bytes) + , watch.elapsedSeconds(), formatReadableQuantity(static_cast(progress.written_rows / watch.elapsedSeconds())) + , formatReadableSizeWithBinarySuffix(static_cast(progress.written_bytes / watch.elapsedSeconds()))); } } @@ -564,9 +593,10 @@ void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr try { + Context query_context = createQueryContext(global_context); String comment = "Materialize MySQL step 2: execute MySQL DDL for sync data"; String event_database = query_event.schema == mysql_database_name ? database_name : ""; - tryToExecuteQuery(query_prefix + query_event.query, global_context, event_database, comment); + tryToExecuteQuery(query_prefix + query_event.query, query_context, event_database, comment); } catch (Exception & exception) { @@ -616,8 +646,9 @@ void MaterializeMySQLSyncThread::Buffers::commit(const Context & context) { for (auto & table_name_and_buffer : data) { + Context query_context = createQueryContext(context); OneBlockInputStream input(table_name_and_buffer.second->first); - BlockOutputStreamPtr out = getTableOutput(database, table_name_and_buffer.first, context); + BlockOutputStreamPtr out = getTableOutput(database, table_name_and_buffer.first, query_context, true); copyData(input, *out); } @@ -640,11 +671,11 @@ MaterializeMySQLSyncThread::Buffers::BufferAndSortingColumnsPtr MaterializeMySQL const auto & iterator = data.find(table_name); if (iterator == data.end()) { - StoragePtr storage = getDatabase(database).tryGetTable(table_name, context); + StoragePtr storage = DatabaseCatalog::instance().getTable(StorageID(database, table_name), context); const StorageInMemoryMetadata & metadata = storage->getInMemoryMetadata(); BufferAndSortingColumnsPtr & buffer_and_soring_columns = data.try_emplace( - table_name, std::make_shared(metadata.getSampleBlockNonMaterialized(), std::vector{})).first->second; + table_name, std::make_shared(metadata.getSampleBlock(), std::vector{})).first->second; Names required_for_sorting_key = metadata.getColumnsRequiredForSortingKey(); diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 31eb3324b94..7bf659e1939 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -339,12 +339,24 @@ ASTs InterpreterCreateImpl::getRewrittenQueries( auto columns = std::make_shared(); + const auto & create_materialized_column_declaration = [&](const String & name, const String & type, const auto & default_value) + { + const auto column_declaration = std::make_shared(); + column_declaration->name = name; + column_declaration->type = makeASTFunction(type); + column_declaration->default_specifier = "MATERIALIZED"; + column_declaration->default_expression = std::make_shared(default_value); + column_declaration->children.emplace_back(column_declaration->type); + column_declaration->children.emplace_back(column_declaration->default_expression); + return column_declaration; + }; + /// Add _sign and _version column. String sign_column_name = getUniqueColumnName(columns_name_and_type, "_sign"); String version_column_name = getUniqueColumnName(columns_name_and_type, "_version"); - columns_name_and_type.emplace_back(NameAndTypePair{sign_column_name, std::make_shared()}); - columns_name_and_type.emplace_back(NameAndTypePair{version_column_name, std::make_shared()}); columns->set(columns->columns, InterpreterCreateQuery::formatColumns(columns_name_and_type)); + columns->columns->children.emplace_back(create_materialized_column_declaration(sign_column_name, "Int8", UInt64(1))); + columns->columns->children.emplace_back(create_materialized_column_declaration(version_column_name, "UInt64", UInt64(1))); auto storage = std::make_shared(); diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index a855fccef56..fc898ba0702 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -22,16 +22,9 @@ namespace DB StorageMaterializeMySQL::StorageMaterializeMySQL(const StoragePtr & nested_storage_, const DatabaseMaterializeMySQL * database_) : IStorage(nested_storage_->getStorageID()), nested_storage(nested_storage_), database(database_) { - ColumnsDescription columns_desc; - const auto & nested_memory_metadata = nested_storage->getInMemoryMetadata(); - const ColumnsDescription & nested_columns_desc = nested_memory_metadata.getColumns(); - - auto iterator = nested_columns_desc.begin(); - for (size_t index = 0; index < nested_columns_desc.size() - 2; ++index, ++iterator) - columns_desc.add(*iterator); - + auto nested_memory_metadata = nested_storage->getInMemoryMetadata(); StorageInMemoryMetadata in_memory_metadata; - in_memory_metadata.setColumns(columns_desc); + in_memory_metadata.setColumns(nested_memory_metadata.getColumns()); setInMemoryMetadata(in_memory_metadata); } @@ -106,17 +99,7 @@ NamesAndTypesList StorageMaterializeMySQL::getVirtuals() const { /// If the background synchronization thread has exception. database->rethrowExceptionIfNeed(); - - NamesAndTypesList virtuals; - Block nested_header = nested_storage->getInMemoryMetadata().getSampleBlockNonMaterialized(); - ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); - ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); - virtuals.emplace_back(NameAndTypePair(sign_column.name, sign_column.type)); - virtuals.emplace_back(NameAndTypePair(version_column.name, version_column.type)); - - auto nested_virtuals = nested_storage->getVirtuals(); - virtuals.insert(virtuals.end(), nested_virtuals.begin(), nested_virtuals.end()); - return virtuals; + return nested_storage->getVirtuals(); } } diff --git a/src/Storages/StorageMaterializeMySQL.h b/src/Storages/StorageMaterializeMySQL.h index 39eca3191f1..d3e132844ee 100644 --- a/src/Storages/StorageMaterializeMySQL.h +++ b/src/Storages/StorageMaterializeMySQL.h @@ -19,7 +19,6 @@ public: bool supportsFinal() const override { return nested_storage->supportsFinal(); } bool supportsSampling() const override { return nested_storage->supportsSampling(); } - StorageMaterializeMySQL(const StoragePtr & nested_storage_, const DatabaseMaterializeMySQL * database_); Pipes read( From f892b7acc01a368a56ceefe52cad4b0debb483c5 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 10 Aug 2020 18:45:09 +0300 Subject: [PATCH 234/374] [docs] improve code blocks padding --- website/css/docs.css | 3 +++ 1 file changed, 3 insertions(+) diff --git a/website/css/docs.css b/website/css/docs.css index f95f08d389c..6be02c6343f 100644 --- a/website/css/docs.css +++ b/website/css/docs.css @@ -69,6 +69,9 @@ summary { padding: 0.125rem 0.25rem; } +#content code.syntax { + padding: 0; +} #content pre { background: #efefef; From e06fd84ca746af536f411b1b66aaf3139a348740 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Mon, 10 Aug 2020 20:16:23 +0300 Subject: [PATCH 235/374] Update synthetic_hardware_benchmark.xml https://github.com/ClickHouse/ClickHouse/pull/13099#issuecomment-671479904 --- tests/performance/synthetic_hardware_benchmark.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/synthetic_hardware_benchmark.xml b/tests/performance/synthetic_hardware_benchmark.xml index 64c9aa05501..5de363b6e20 100644 --- a/tests/performance/synthetic_hardware_benchmark.xml +++ b/tests/performance/synthetic_hardware_benchmark.xml @@ -1,4 +1,4 @@ - + 30000000000 From d42dbfce987a7510bb69139eb6e8e099a3f3c0df Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 10 Aug 2020 23:36:52 +0300 Subject: [PATCH 236/374] Do not overallocate when creating PODArray with given size Another part of #12278 --- src/Common/PODArray.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index b88e9e33ed4..1084f0800cc 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -111,7 +111,7 @@ protected: void alloc_for_num_elements(size_t num_elements) { - alloc(roundUpToPowerOfTwoOrZero(minimum_memory_for_elements(num_elements))); + alloc(minimum_memory_for_elements(num_elements)); } template From 42ecf91df1686d12957af075c4cd6e070c0a09aa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 11 Aug 2020 00:36:49 +0300 Subject: [PATCH 237/374] Fix "Arcadia" build --- programs/client/ConnectionParameters.cpp | 7 ++++++- programs/ya.make | 5 ----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/programs/client/ConnectionParameters.cpp b/programs/client/ConnectionParameters.cpp index f0ef3ae5694..19734dd5ffa 100644 --- a/programs/client/ConnectionParameters.cpp +++ b/programs/client/ConnectionParameters.cpp @@ -9,7 +9,10 @@ #include #include #include -#include + +#if !defined(ARCADIA_BUILD) +#include // Y_IGNORE +#endif namespace DB { @@ -49,10 +52,12 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati } if (password_prompt) { +#if !defined(ARCADIA_BUILD) std::string prompt{"Password for user (" + user + "): "}; char buf[1000] = {}; if (auto * result = readpassphrase(prompt.c_str(), buf, sizeof(buf), 0)) password = result; +#endif } compression = config.getBool("compression", true) ? Protocol::Compression::Enable : Protocol::Compression::Disable; diff --git a/programs/ya.make b/programs/ya.make index 31e7b6f8569..1b80b264959 100644 --- a/programs/ya.make +++ b/programs/ya.make @@ -6,14 +6,9 @@ CFLAGS( -DENABLE_CLICKHOUSE_SERVER ) -ADDINCL ( - GLOBAL clickhouse/base/readpassphrase -) - PEERDIR( clickhouse/base/daemon clickhouse/base/loggers - clickhouse/base/readpassphrase clickhouse/src ) From c76ab5340f287086b8eec998debaa7280e665372 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 4 Aug 2020 23:08:23 +0300 Subject: [PATCH 238/374] Add csv_with_names prefix to 01375_storage_file_write_prefix test --- ...e => 01375_storage_file_write_prefix_csv_with_names.reference} | 0 ...fix.sql => 01375_storage_file_write_prefix_csv_with_names.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01375_storage_file_write_prefix.reference => 01375_storage_file_write_prefix_csv_with_names.reference} (100%) rename tests/queries/0_stateless/{01375_storage_file_write_prefix.sql => 01375_storage_file_write_prefix_csv_with_names.sql} (100%) diff --git a/tests/queries/0_stateless/01375_storage_file_write_prefix.reference b/tests/queries/0_stateless/01375_storage_file_write_prefix_csv_with_names.reference similarity index 100% rename from tests/queries/0_stateless/01375_storage_file_write_prefix.reference rename to tests/queries/0_stateless/01375_storage_file_write_prefix_csv_with_names.reference diff --git a/tests/queries/0_stateless/01375_storage_file_write_prefix.sql b/tests/queries/0_stateless/01375_storage_file_write_prefix_csv_with_names.sql similarity index 100% rename from tests/queries/0_stateless/01375_storage_file_write_prefix.sql rename to tests/queries/0_stateless/01375_storage_file_write_prefix_csv_with_names.sql From f18b06f7fa2192620e169557fbe6a126a91c4bc8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 4 Aug 2020 23:10:06 +0300 Subject: [PATCH 239/374] Add test for File(TSVWithNames) --- ...file_write_prefix_tsv_with_names.reference | 30 +++++++++++++++++++ ...orage_file_write_prefix_tsv_with_names.sql | 14 +++++++++ 2 files changed, 44 insertions(+) create mode 100644 tests/queries/0_stateless/01375_storage_file_write_prefix_tsv_with_names.reference create mode 100644 tests/queries/0_stateless/01375_storage_file_write_prefix_tsv_with_names.sql diff --git a/tests/queries/0_stateless/01375_storage_file_write_prefix_tsv_with_names.reference b/tests/queries/0_stateless/01375_storage_file_write_prefix_tsv_with_names.reference new file mode 100644 index 00000000000..ed9a18b9346 --- /dev/null +++ b/tests/queries/0_stateless/01375_storage_file_write_prefix_tsv_with_names.reference @@ -0,0 +1,30 @@ +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 diff --git a/tests/queries/0_stateless/01375_storage_file_write_prefix_tsv_with_names.sql b/tests/queries/0_stateless/01375_storage_file_write_prefix_tsv_with_names.sql new file mode 100644 index 00000000000..55a97eb6e95 --- /dev/null +++ b/tests/queries/0_stateless/01375_storage_file_write_prefix_tsv_with_names.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS tmp_01375; +DROP TABLE IF EXISTS table_tsv_01375; + +CREATE TABLE tmp_01375 (n UInt32, s String) ENGINE = Memory; +CREATE TABLE table_tsv_01375 AS tmp_01375 ENGINE = File(TSVWithNames); + +INSERT INTO table_tsv_01375 SELECT number as n, toString(n) as s FROM numbers(10); +INSERT INTO table_tsv_01375 SELECT number as n, toString(n) as s FROM numbers(10); +INSERT INTO table_tsv_01375 SELECT number as n, toString(n) as s FROM numbers(10); + +SELECT * FROM table_tsv_01375; + +DROP TABLE IF EXISTS tmp_01375; +DROP TABLE IF EXISTS table_tsv_01375; From ba92213d21ac8e1f45342bdb9174e8182658ef61 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 4 Aug 2020 23:49:30 +0300 Subject: [PATCH 240/374] Write header only once in TSVWithNames* --- src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index 1c493cd1ab4..95377c13b55 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -18,7 +18,7 @@ TabSeparatedRowOutputFormat::TabSeparatedRowOutputFormat( } -void TabSeparatedRowOutputFormat::writePrefix() +void TabSeparatedRowOutputFormat::doWritePrefix() { const auto & header = getPort(PortKind::Main).getHeader(); size_t columns = header.columns(); diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h index 3d3f82e7833..137ffde4b18 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h @@ -31,10 +31,11 @@ public: void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; void writeFieldDelimiter() override; void writeRowEndDelimiter() override; - void writePrefix() override; void writeBeforeTotals() override; void writeBeforeExtremes() override; + void doWritePrefix() override; + /// https://www.iana.org/assignments/media-types/text/tab-separated-values String getContentType() const override { return "text/tab-separated-values; charset=UTF-8"; } From d5a2e1e61cc6d86f163519e131591678195be349 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 5 Aug 2020 00:09:36 +0300 Subject: [PATCH 241/374] Call doWritePrefix() if IOutputFormat works as Processor (i.e. clickhouse-local) This fixes the *WithNames output format (lack of header before this patch). --- src/Processors/Formats/IOutputFormat.cpp | 6 ++++++ src/Processors/Formats/IOutputFormat.h | 2 ++ 2 files changed, 8 insertions(+) diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index f7fc6170cad..76920c0fa53 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -46,6 +46,12 @@ IOutputFormat::Status IOutputFormat::prepare() void IOutputFormat::work() { + if (!prefix_written) + { + doWritePrefix(); + prefix_written = true; + } + if (finished && !finalized) { if (rows_before_limit_counter && rows_before_limit_counter->hasAppliedLimit()) diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index bab746c1772..67c307df2aa 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -87,6 +87,8 @@ private: /// Counters for consumed chunks. Are used for QueryLog. size_t result_rows = 0; size_t result_bytes = 0; + + bool prefix_written = false; }; } From 8d5011a3e72de2a0d9ada3799ef58cf6dae635fd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 5 Aug 2020 00:12:39 +0300 Subject: [PATCH 242/374] Add a test for *WithNames* with clickhouse-local --- ..._output_format_tsv_csv_with_names.reference | 13 +++++++++++++ .../01375_output_format_tsv_csv_with_names.sh | 18 ++++++++++++++++++ 2 files changed, 31 insertions(+) create mode 100644 tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.reference create mode 100755 tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.sh diff --git a/tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.reference b/tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.reference new file mode 100644 index 00000000000..6f1974ccd73 --- /dev/null +++ b/tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.reference @@ -0,0 +1,13 @@ +TSVWithNames +number +0 +1 +TSVWithNamesAndTypes +number +UInt64 +0 +1 +CSVWithNames +"number" +0 +1 diff --git a/tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.sh b/tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.sh new file mode 100755 index 00000000000..de4486a88a5 --- /dev/null +++ b/tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +opts=( + --input-format CSV + -q 'SELECT number FROM numbers(2)' +) + +echo 'TSVWithNames' +${CLICKHOUSE_LOCAL} "${opts[@]}" --format TSVWithNames + +echo 'TSVWithNamesAndTypes' +${CLICKHOUSE_LOCAL} "${opts[@]}" --format TSVWithNamesAndTypes + +echo 'CSVWithNames' +${CLICKHOUSE_LOCAL} "${opts[@]}" --format CSVWithNames From e43f49edfbca9344965dd0bd8ebb218e7cbca6e2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 11 Aug 2020 00:51:53 +0300 Subject: [PATCH 243/374] Add a test for *WithNames* via clickhouse-local (corner cases) - empty rows - multiple runs to the same file (write header only once) --- ..._tsv_csv_with_names_write_prefix.reference | 17 +++++++++++ ...ge_file_tsv_csv_with_names_write_prefix.sh | 30 +++++++++++++++++++ 2 files changed, 47 insertions(+) create mode 100644 tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.reference create mode 100755 tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh diff --git a/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.reference b/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.reference new file mode 100644 index 00000000000..4f2a79b9905 --- /dev/null +++ b/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.reference @@ -0,0 +1,17 @@ +zero rows +TSVWithNames +TSVWithNamesAndTypes +CSVWithNames +multi clickhouse-local one file +TSVWithNames +0 +0 +0 +TSVWithNamesAndTypes +0 +0 +0 +CSVWithNames +0 +0 +0 diff --git a/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh b/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh new file mode 100755 index 00000000000..d396981f873 --- /dev/null +++ b/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +# zero rows +echo 'zero rows' +for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames; do + echo $format + ${CLICKHOUSE_LOCAL} --query=" + CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format.tsv') AS SELECT * FROM numbers(1) WHERE number < 0; + SELECT * FROM ${format}_01375; + DROP TABLE ${format}_01375; + " + rm 01375_$format.tsv +done + +# run multiple times to the same file +echo 'multi clickhouse-local one file' +for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames; do + echo $format + for _ in {1..2}; do + ${CLICKHOUSE_LOCAL} --query=" + CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format.tsv') AS SELECT * FROM numbers(1); + SELECT * FROM ${format}_01375; + DROP TABLE ${format}_01375; + " + done + rm 01375_$format.tsv +done From 72205cd326e446eb5939d0250c37950b456d5197 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 11 Aug 2020 00:51:53 +0300 Subject: [PATCH 244/374] Call writePrefix() from ctor in SinkToOutputStream This will fix lack of headers for the following formats if there are zero rows: - TSVWithNames - CSVWithNames - TSVWithNamesAndTypes --- src/Processors/Sources/SinkToOutputStream.cpp | 6 +----- src/Processors/Sources/SinkToOutputStream.h | 1 - 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Processors/Sources/SinkToOutputStream.cpp b/src/Processors/Sources/SinkToOutputStream.cpp index 29619dff463..9727b637d8b 100644 --- a/src/Processors/Sources/SinkToOutputStream.cpp +++ b/src/Processors/Sources/SinkToOutputStream.cpp @@ -8,15 +8,11 @@ SinkToOutputStream::SinkToOutputStream(BlockOutputStreamPtr stream_) : ISink(stream_->getHeader()) , stream(std::move(stream_)) { + stream->writePrefix(); } void SinkToOutputStream::consume(Chunk chunk) { - if (!initialized) - stream->writePrefix(); - - initialized = true; - stream->write(getPort().getHeader().cloneWithColumns(chunk.detachColumns())); } diff --git a/src/Processors/Sources/SinkToOutputStream.h b/src/Processors/Sources/SinkToOutputStream.h index 037cee0085d..5362608551f 100644 --- a/src/Processors/Sources/SinkToOutputStream.h +++ b/src/Processors/Sources/SinkToOutputStream.h @@ -22,7 +22,6 @@ protected: private: BlockOutputStreamPtr stream; - bool initialized = false; }; } From 38cad2e258440806c7393816188a1d5209666d1b Mon Sep 17 00:00:00 2001 From: melin Date: Tue, 11 Aug 2020 09:43:28 +0800 Subject: [PATCH 245/374] fixbug Nullable(String) --- utils/grammar/ClickHouseParser.g4 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/grammar/ClickHouseParser.g4 b/utils/grammar/ClickHouseParser.g4 index 5cb4676fcb8..c2b4a5f6074 100644 --- a/utils/grammar/ClickHouseParser.g4 +++ b/utils/grammar/ClickHouseParser.g4 @@ -185,7 +185,7 @@ clickhouse_type | T_AGGREGATE_FUNCTION LPAREN function_name ( COMMA clickhouse_type ) * RPAREN | T_ARRAY LPAREN clickhouse_type RPAREN | T_TUPLE LPAREN clickhouse_type ( COMMA clickhouse_type ) * RPAREN - | T_NULLABLE LPAREN clickhouse_type LPAREN + | T_NULLABLE LPAREN clickhouse_type RPAREN ; simple_type From 7371cf6bd1bd85546e91247f13883cb8526cf629 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 11 Aug 2020 09:09:08 +0300 Subject: [PATCH 246/374] Proper remote host checking in S3 redirects (#13404) * Proper remote host checking in S3 redirects. Co-authored-by: Alexander Kazakov --- src/Disks/S3/registerDiskS3.cpp | 3 ++- src/IO/S3/PocoHTTPClient.cpp | 14 +++++++++++++- src/IO/S3/PocoHTTPClient.h | 13 ++++++++++++- src/IO/S3/PocoHTTPClientFactory.cpp | 2 +- src/IO/S3Common.cpp | 16 ++++++++++------ src/IO/S3Common.h | 13 +++++++++---- src/Storages/StorageS3.cpp | 2 +- 7 files changed, 48 insertions(+), 15 deletions(-) diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 3500e909e8c..341ada59631 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -128,7 +128,8 @@ void registerDiskS3(DiskFactory & factory) cfg, uri.is_virtual_hosted_style, config.getString(config_prefix + ".access_key_id", ""), - config.getString(config_prefix + ".secret_access_key", "")); + config.getString(config_prefix + ".secret_access_key", ""), + context.getRemoteHostFilter()); String metadata_path = config.getString(config_prefix + ".metadata_path", context.getPath() + "disks/" + name + "/"); diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index efa402c9447..e068f3581bd 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -38,13 +38,24 @@ namespace DB::ErrorCodes namespace DB::S3 { -PocoHTTPClient::PocoHTTPClient(const Aws::Client::ClientConfiguration & clientConfiguration) + +PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( + const Aws::Client::ClientConfiguration & cfg, + const RemoteHostFilter & remote_host_filter_) + : Aws::Client::ClientConfiguration(cfg) + , remote_host_filter(remote_host_filter_) +{ +} + + +PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & clientConfiguration) : per_request_configuration(clientConfiguration.perRequestConfiguration) , timeouts(ConnectionTimeouts( Poco::Timespan(clientConfiguration.connectTimeoutMs * 1000), /// connection timeout. Poco::Timespan(clientConfiguration.httpRequestTimeoutMs * 1000), /// send timeout. Poco::Timespan(clientConfiguration.httpRequestTimeoutMs * 1000) /// receive timeout. )) + , remote_host_filter(clientConfiguration.remote_host_filter) { } @@ -199,6 +210,7 @@ void PocoHTTPClient::MakeRequestInternal( if (poco_response.getStatus() == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT) { auto location = poco_response.get("location"); + remote_host_filter.checkURL(Poco::URI(location)); uri = location; LOG_DEBUG(log, "Redirecting request to new location: {}", location); diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 203ad94e86b..a94a08e217d 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -1,7 +1,10 @@ #pragma once +#include #include +#include #include +#include namespace Aws::Http::Standard { @@ -11,10 +14,17 @@ class StandardHttpResponse; namespace DB::S3 { +struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration +{ + const RemoteHostFilter & remote_host_filter; + + PocoHTTPClientConfiguration(const Aws::Client::ClientConfiguration & cfg, const RemoteHostFilter & remote_host_filter_); +}; + class PocoHTTPClient : public Aws::Http::HttpClient { public: - explicit PocoHTTPClient(const Aws::Client::ClientConfiguration & clientConfiguration); + explicit PocoHTTPClient(const PocoHTTPClientConfiguration & clientConfiguration); ~PocoHTTPClient() override = default; std::shared_ptr MakeRequest( Aws::Http::HttpRequest & request, @@ -35,6 +45,7 @@ private: std::function per_request_configuration; ConnectionTimeouts timeouts; + const RemoteHostFilter & remote_host_filter; }; } diff --git a/src/IO/S3/PocoHTTPClientFactory.cpp b/src/IO/S3/PocoHTTPClientFactory.cpp index e4b86593ec1..68f6a6b9823 100644 --- a/src/IO/S3/PocoHTTPClientFactory.cpp +++ b/src/IO/S3/PocoHTTPClientFactory.cpp @@ -11,7 +11,7 @@ namespace DB::S3 std::shared_ptr PocoHTTPClientFactory::CreateHttpClient(const Aws::Client::ClientConfiguration & clientConfiguration) const { - return std::make_shared(clientConfiguration); + return std::make_shared(static_cast(clientConfiguration)); } std::shared_ptr PocoHTTPClientFactory::CreateHttpRequest( diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 33553c5f264..ba9a61ab922 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -166,25 +166,27 @@ namespace S3 const String & endpoint, bool is_virtual_hosted_style, const String & access_key_id, - const String & secret_access_key) + const String & secret_access_key, + const RemoteHostFilter & remote_host_filter) { Aws::Client::ClientConfiguration cfg; if (!endpoint.empty()) cfg.endpointOverride = endpoint; - return create(cfg, is_virtual_hosted_style, access_key_id, secret_access_key); + return create(cfg, is_virtual_hosted_style, access_key_id, secret_access_key, remote_host_filter); } std::shared_ptr ClientFactory::create( // NOLINT Aws::Client::ClientConfiguration & cfg, bool is_virtual_hosted_style, const String & access_key_id, - const String & secret_access_key) + const String & secret_access_key, + const RemoteHostFilter & remote_host_filter) { Aws::Auth::AWSCredentials credentials(access_key_id, secret_access_key); - Aws::Client::ClientConfiguration client_configuration = cfg; + PocoHTTPClientConfiguration client_configuration(cfg, remote_host_filter); if (!client_configuration.endpointOverride.empty()) { @@ -214,9 +216,11 @@ namespace S3 bool is_virtual_hosted_style, const String & access_key_id, const String & secret_access_key, - HeaderCollection headers) + HeaderCollection headers, + const RemoteHostFilter & remote_host_filter) { - Aws::Client::ClientConfiguration cfg; + PocoHTTPClientConfiguration cfg({}, remote_host_filter); + if (!endpoint.empty()) cfg.endpointOverride = endpoint; diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 9f6b93ca6c7..45ec0ad90c6 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -5,8 +5,9 @@ #if USE_AWS_S3 #include -#include #include +#include +#include namespace Aws::S3 { @@ -15,6 +16,7 @@ namespace Aws::S3 namespace DB { + class RemoteHostFilter; struct HttpHeader; using HeaderCollection = std::vector; } @@ -33,20 +35,23 @@ public: const String & endpoint, bool is_virtual_hosted_style, const String & access_key_id, - const String & secret_access_key); + const String & secret_access_key, + const RemoteHostFilter & remote_host_filter); std::shared_ptr create( Aws::Client::ClientConfiguration & cfg, bool is_virtual_hosted_style, const String & access_key_id, - const String & secret_access_key); + const String & secret_access_key, + const RemoteHostFilter & remote_host_filter); std::shared_ptr create( const String & endpoint, bool is_virtual_hosted_style, const String & access_key_id, const String & secret_access_key, - HeaderCollection headers); + HeaderCollection headers, + const RemoteHostFilter & remote_host_filter); private: ClientFactory(); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index fd27229e530..e7bce92806c 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -218,7 +218,7 @@ StorageS3::StorageS3( credentials = Aws::Auth::AWSCredentials(std::move(settings.access_key_id), std::move(settings.secret_access_key)); client = S3::ClientFactory::instance().create( - uri_.endpoint, uri_.is_virtual_hosted_style, access_key_id_, secret_access_key_, std::move(settings.headers)); + uri_.endpoint, uri_.is_virtual_hosted_style, access_key_id_, secret_access_key_, std::move(settings.headers), context_.getRemoteHostFilter()); } From 93d73c78c93a32be8e4cca0fec78c1e45e0e03a1 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 11 Aug 2020 17:03:43 +0800 Subject: [PATCH 247/374] ISSUES-4006 resolve review comment & add test case --- .../MySQL/InterpretersMySQLDDLQuery.cpp | 13 +- .../MySQL/tests/gtest_create_rewritten.cpp | 122 ++++++++++++++++-- src/Storages/StorageMaterializeMySQL.cpp | 2 +- 3 files changed, 118 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 7bf659e1939..9e94e8f579e 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -201,11 +201,11 @@ static String getUniqueColumnName(NamesAndTypesList columns_name_and_type, const static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys) { - const auto & numbers_partition = [&](const String & column_name, const DataTypePtr & type, size_t type_max_size) + const auto & numbers_partition = [&](const String & column_name, bool is_nullable, size_t type_max_size) { ASTPtr column = std::make_shared(column_name); - if (type->isNullable()) + if (is_nullable) column = makeASTFunction("assumeNotNull", column); return makeASTFunction("intDiv", column, std::make_shared(UInt64(type_max_size / 1000))); @@ -237,21 +237,24 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys) { best_size = type->getSizeOfValueInMemory(); best_partition = std::make_shared(primary_key.name); + + if (primary_key.type->isNullable()) + best_partition = makeASTFunction("assumeNotNull", best_partition); } else if (which.isInt16() || which.isUInt16()) { best_size = type->getSizeOfValueInMemory(); - best_partition = numbers_partition(primary_key.name, type, std::numeric_limits::max()); + best_partition = numbers_partition(primary_key.name, primary_key.type->isNullable(), std::numeric_limits::max()); } else if (which.isInt32() || which.isUInt32()) { best_size = type->getSizeOfValueInMemory(); - best_partition = numbers_partition(primary_key.name, type, std::numeric_limits::max()); + best_partition = numbers_partition(primary_key.name, primary_key.type->isNullable(), std::numeric_limits::max()); } else if (which.isInt64() || which.isUInt64()) { best_size = type->getSizeOfValueInMemory(); - best_partition = numbers_partition(primary_key.name, type, std::numeric_limits::max()); + best_partition = numbers_partition(primary_key.name, primary_key.type->isNullable(), std::numeric_limits::max()); } } } diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index c8aa2d59b3f..b9bfe28ea1b 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -19,40 +19,136 @@ using namespace DB; static inline ASTPtr tryRewrittenCreateQuery(const String & query, const Context & context) { ParserExternalDDLQuery external_ddl_parser; - ASTPtr ast = parseQuery(external_ddl_parser, query, 0, 0); + ASTPtr ast = parseQuery(external_ddl_parser, "EXTERNAL DDL FROM MySQL(test_database, test_database) " + query, 0, 0); return MySQLInterpreter::InterpreterCreateImpl::getRewrittenQueries( *ast->as()->external_ddl->as(), context, "test_database", "test_database")[0]; } +TEST(MySQLCreateRewritten, ColumnsDataType) +{ + tryRegisterFunctions(); + const auto & context_holder = getContext(); + + std::vector> test_types + { + {"TINYINT", "Int8"}, {"SMALLINT", "Int16"}, {"MEDIUMINT", "Int32"}, {"INT", "Int32"}, + {"INTEGER", "Int32"}, {"BIGINT", "Int64"}, {"FLOAT", "Float32"}, {"DOUBLE", "Float64"}, + {"VARCHAR(10)", "String"}, {"CHAR(10)", "String"}, {"Date", "Date"}, {"DateTime", "DateTime"}, + {"TIMESTAMP", "DateTime"}, {"BOOLEAN", "Int8"} + }; + + for (const auto & [test_type, mapped_type] : test_types) + { + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + ")", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` Nullable(" + mapped_type + ")" + ", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); + + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " NOT NULL)", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` " + mapped_type + + ", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); + + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " COMMENT 'test_comment' NOT NULL)", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` " + mapped_type + + ", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); + + if (Poco::toUpper(test_type).find("INT") != std::string::npos) + { + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " UNSIGNED)", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` Nullable(U" + mapped_type + ")" + ", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); + + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " COMMENT 'test_comment' UNSIGNED)", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` Nullable(U" + mapped_type + ")" + ", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); + + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " NOT NULL UNSIGNED)", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` U" + mapped_type + + ", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); + + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " COMMENT 'test_comment' UNSIGNED NOT NULL)", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` U" + mapped_type + + ", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); + } + } +} + +TEST(MySQLCreateRewritten, PartitionPolicy) +{ + tryRegisterFunctions(); + const auto & context_holder = getContext(); + + std::vector> test_types + { + {"TINYINT", "Int8", " PARTITION BY key"}, {"SMALLINT", "Int16", " PARTITION BY intDiv(key, 65)"}, + {"MEDIUMINT", "Int32", " PARTITION BY intDiv(key, 4294967)"}, {"INT", "Int32", " PARTITION BY intDiv(key, 4294967)"}, + {"INTEGER", "Int32", " PARTITION BY intDiv(key, 4294967)"}, {"BIGINT", "Int64", " PARTITION BY intDiv(key, 18446744073709551)"}, + {"FLOAT", "Float32", ""}, {"DOUBLE", "Float64", ""}, {"VARCHAR(10)", "String", ""}, {"CHAR(10)", "String", ""}, + {"Date", "Date", " PARTITION BY toYYYYMM(key)"}, {"DateTime", "DateTime", " PARTITION BY toYYYYMM(key)"}, + {"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Int8", " PARTITION BY key"} + }; + + const auto & replace_string = [](const String & str, const String & old_str, const String & new_str) + { + String new_string = str; + size_t pos = new_string.find(old_str); + if (pos != std::string::npos) + new_string = new_string.replace(pos, old_str.size(), new_str); + return new_string; + }; + + for (const auto & [test_type, mapped_type, partition_policy] : test_types) + { + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " PRIMARY KEY)", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` Nullable(" + mapped_type + "), `_sign` Int8() MATERIALIZED 1, " + "`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + replace_string(partition_policy, "key", "assumeNotNull(key)") + " ORDER BY tuple(key)"); + + EXPECT_EQ(queryToString(tryRewrittenCreateQuery( + "CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " NOT NULL PRIMARY KEY)", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `_sign` Int8() MATERIALIZED 1, " + "`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY tuple(key)"); + } +} + TEST(MySQLCreateRewritten, RewrittenQueryWithPrimaryKey) { tryRegisterFunctions(); const auto & context_holder = getContext(); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( - "EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL PRIMARY " - "KEY) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` Int32, `_sign` Int8, `_version` UInt64) ENGINE = ReplacingMergeTree(_version) " + "CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL PRIMARY KEY) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` Int32, `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version) " "PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( - "EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL, " - " PRIMARY KEY (`key`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` Int32, `_sign` Int8, `_version` UInt64) ENGINE = ReplacingMergeTree(_version) " + "CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL, PRIMARY KEY (`key`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key` Int32, `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version) " "PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( - "EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key_1` int NOT NULL, " - " key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key_1` Int32, `key_2` Int32, `_sign` Int8, `_version` UInt64) ENGINE = " + "CREATE TABLE `test_database`.`test_table_1` (`key_1` int NOT NULL, key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key_1` Int32, `key_2` Int32, `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " "ReplacingMergeTree(_version) PARTITION BY intDiv(key_1, 4294967) ORDER BY (key_1, key_2)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( - "EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key_1` BIGINT NOT NULL, " - " key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key_1` Int64, `key_2` Int32, `_sign` Int8, `_version` UInt64) ENGINE = " + "CREATE TABLE `test_database`.`test_table_1` (`key_1` BIGINT NOT NULL, key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), + "CREATE TABLE test_database.test_table_1 (`key_1` Int64, `key_2` Int32, `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " "ReplacingMergeTree(_version) PARTITION BY intDiv(key_2, 4294967) ORDER BY (key_1, key_2)"); } diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index fc898ba0702..92793df22a0 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -43,7 +43,7 @@ Pipes StorageMaterializeMySQL::read( NameSet column_names_set = NameSet(column_names.begin(), column_names.end()); const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr(); - Block nested_header = nested_metadata->getSampleBlockNonMaterialized(); + Block nested_header = nested_metadata->getSampleBlock(); ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2); ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1); From 5a97f0c19e3990d96805b9460c93064b713a0fe2 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 11 Aug 2020 14:53:25 +0300 Subject: [PATCH 248/374] performance comparison --- docker/test/performance-comparison/compare.sh | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index c916fe4a639..f80fef45a64 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -318,10 +318,10 @@ create view right_query_log as select * create view query_logs as select 0 version, query_id, ProfileEvents.Names, ProfileEvents.Values, - query_duration_ms from left_query_log + query_duration_ms, memory_usage from left_query_log union all select 1 version, query_id, ProfileEvents.Names, ProfileEvents.Values, - query_duration_ms from right_query_log + query_duration_ms, memory_usage from right_query_log ; -- This is a single source of truth on all metrics we have for query runs. The @@ -345,10 +345,11 @@ create table query_run_metric_arrays engine File(TSV, 'analyze/query-run-metric- arrayMap(x->toFloat64(x), ProfileEvents.Values))] ), arrayReduce('sumMapState', [( - ['client_time', 'server_time'], + ['client_time', 'server_time', 'memory_usage'], arrayMap(x->if(x != 0., x, -0.), [ toFloat64(query_runs.time), - toFloat64(query_duration_ms / 1000.)]))]) + toFloat64(query_duration_ms / 1000.), + toFloat64(memory_usage)]))]) ] )) as metrics_tuple).1 metric_names, metrics_tuple.2 metric_values From 5b677e1536855197665af7530cfb48fc43dbdc24 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Tue, 11 Aug 2020 14:09:36 +0200 Subject: [PATCH 249/374] Incresing health-check timeouts for clickhouse nodes in the all the modules. Adding support to dump docker-compose logs if unhealthy containers found. --- .../example/docker-compose/clickhouse-service.yml | 2 +- tests/testflows/helpers/cluster.py | 8 +++++--- .../testflows/ldap/docker-compose/clickhouse-service.yml | 2 +- .../testflows/rbac/docker-compose/clickhouse-service.yml | 2 +- 4 files changed, 8 insertions(+), 6 deletions(-) diff --git a/tests/testflows/example/docker-compose/clickhouse-service.yml b/tests/testflows/example/docker-compose/clickhouse-service.yml index ed345bd7b04..36f13e40c5c 100644 --- a/tests/testflows/example/docker-compose/clickhouse-service.yml +++ b/tests/testflows/example/docker-compose/clickhouse-service.yml @@ -20,7 +20,7 @@ services: test: clickhouse client --query='select 1' interval: 3s timeout: 5s - retries: 40 + retries: 100 start_period: 2s cap_add: - SYS_PTRACE diff --git a/tests/testflows/helpers/cluster.py b/tests/testflows/helpers/cluster.py index aa39c268d2f..660e831ffb8 100644 --- a/tests/testflows/helpers/cluster.py +++ b/tests/testflows/helpers/cluster.py @@ -246,8 +246,8 @@ class Cluster(object): assert os.path.exists(self.clickhouse_binary_path) os.environ["CLICKHOUSE_TESTS_SERVER_BIN_PATH"] = self.clickhouse_binary_path - os.environ["CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH"] = os.path.join(os.path.dirname(self.clickhouse_binary_path), - "clickhouse-odbc-bridge") + os.environ["CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH"] = os.path.join( + os.path.dirname(self.clickhouse_binary_path), "clickhouse-odbc-bridge") os.environ["CLICKHOUSE_TESTS_DIR"] = self.configs_dir with Given("docker-compose"): @@ -258,7 +258,9 @@ class Cluster(object): cmd = self.command(None, f'{self.docker_compose} up -d --no-recreate 2>&1 | tee') with Then("check there are no unhealthy containers"): - assert "is unhealthy" not in cmd.output, error() + if "is unhealthy" in cmd.output: + self.command(None, f'{self.docker_compose} logs | tee') + fail("found unhealthy containers") with Then("wait all nodes report healhy"): for name in self.nodes["clickhouse"]: diff --git a/tests/testflows/ldap/docker-compose/clickhouse-service.yml b/tests/testflows/ldap/docker-compose/clickhouse-service.yml index e6fa8542948..12553d7948d 100644 --- a/tests/testflows/ldap/docker-compose/clickhouse-service.yml +++ b/tests/testflows/ldap/docker-compose/clickhouse-service.yml @@ -20,7 +20,7 @@ services: test: clickhouse client --query='select 1' interval: 3s timeout: 10s - retries: 5 + retries: 100 start_period: 30s cap_add: - SYS_PTRACE diff --git a/tests/testflows/rbac/docker-compose/clickhouse-service.yml b/tests/testflows/rbac/docker-compose/clickhouse-service.yml index 1e9f48b9b45..a41f0da2839 100644 --- a/tests/testflows/rbac/docker-compose/clickhouse-service.yml +++ b/tests/testflows/rbac/docker-compose/clickhouse-service.yml @@ -20,7 +20,7 @@ services: test: clickhouse client --query='select 1' interval: 3s timeout: 2s - retries: 40 + retries: 100 start_period: 2s cap_add: - SYS_PTRACE From e3b51835ff32f6ce03914bffd3de6c6ac28f27ff Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 11 Aug 2020 21:35:50 +0800 Subject: [PATCH 250/374] ISSUES-4006 try fix ci --- .../MySQL/InterpretersMySQLDDLQuery.cpp | 2 +- .../materialize_with_ddl.py | 37 +++++++++---------- 2 files changed, 19 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 9e94e8f579e..9320e8eacf5 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -351,7 +351,7 @@ ASTs InterpreterCreateImpl::getRewrittenQueries( column_declaration->default_expression = std::make_shared(default_value); column_declaration->children.emplace_back(column_declaration->type); column_declaration->children.emplace_back(column_declaration->default_expression); - return column_declaration; + return std::move(column_declaration); }; /// Add _sign and _version column. diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index dd3586a4624..cc3a8f82fe1 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -177,8 +177,7 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_1 INT NOT NULL") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") - mysql_node.query( - "ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ("0" if service_name == "mysql5_7" else "(id)")) + mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ("0" if service_name == "mysql5_7" else "(id)")) # create mapping clickhouse_node.query( @@ -187,10 +186,10 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", - "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t\t\t\t\t\n") + "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query( "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( @@ -198,7 +197,7 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node default_expression = "DEFAULT\t0" if service_name == "mysql5_7" else "DEFAULT\tid" check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", - "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t" + default_expression + "\t\t\t\n") + "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t" + default_expression + "\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2, 3, 4, 5), (6, 7, 8, 9, 10)") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\t2\t3\t4\t5\n6\t7\t8\t9\t10\n") @@ -219,12 +218,12 @@ def alter_drop_column_with_materialize_mysql_database(clickhouse_node, mysql_nod assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\ndrop_column\tInt32\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\ndrop_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_2 DROP COLUMN drop_column") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1), (2), (3), (4), (5)") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") @@ -247,12 +246,12 @@ def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_n assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nrename_column\tInt32\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nrename_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_2 RENAME COLUMN rename_column TO new_column_name") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2), (3, 4), (5, 6), (7, 8), (9, 10)") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\t4\n5\t6\n7\t8\n9\t10\n") @@ -275,16 +274,16 @@ def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_n assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tInt32\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT FIRST") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "modify_column\tNullable(Int32)\t\t\t\t\t\nid\tInt32\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "modify_column\tNullable(Int32)\t\t\t\t\t\nid\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT AFTER id") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2), (3, NULL)") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\t\\N\n") @@ -309,13 +308,13 @@ def alter_rename_table_with_materialize_mysql_database(clickhouse_node, mysql_no assert "test_database" in clickhouse_node.query("SHOW DATABASES") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_3\n") - check_query(clickhouse_node, "DESC test_database.test_table_3 FORMAT TSV", "id\tInt32\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_3 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_3\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\ndrop_column\tInt32\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\ndrop_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column, RENAME TO test_database.test_table_2, RENAME TO test_database.test_table_4") check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_3\ntest_table_4\n") - check_query(clickhouse_node, "DESC test_database.test_table_4 FORMAT TSV", "id\tInt32\t\t\t\t\t\n") + check_query(clickhouse_node, "DESC test_database.test_table_4 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query("INSERT INTO test_database.test_table_4 VALUES(1), (2), (3), (4), (5)") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_4 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") From f18c1ce03e72199d065d008b4e73731d44d3274e Mon Sep 17 00:00:00 2001 From: roman Date: Tue, 11 Aug 2020 14:55:36 +0100 Subject: [PATCH 251/374] [docs]: explicitly mention unit of measurement for `max_insert_block_size` Without explicitly defined unit of measurement the description for setting `max_insert_block_size` may result into confusion and mistakenly interpret as bytes. --- docs/en/operations/settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 27d5a2e6f79..8b301de8ede 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -654,7 +654,7 @@ log_query_threads=1 ## max\_insert\_block\_size {#settings-max_insert_block_size} -The size of blocks to form for insertion into a table. +The size of blocks (in a count of rows) to form for insertion into a table. This setting only applies in cases when the server forms the blocks. For example, for an INSERT via the HTTP interface, the server parses the data format and forms blocks of the specified size. But when using clickhouse-client, the client parses the data itself, and the ‘max\_insert\_block\_size’ setting on the server doesn’t affect the size of the inserted blocks. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 589b88597c8..97e962ffe0d 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -571,7 +571,7 @@ log_query_threads=1 ## max\_insert\_block\_size {#settings-max_insert_block_size} -Формировать блоки указанного размера, при вставке в таблицу. +Формировать блоки указанного размера (в количестве строк), при вставке в таблицу. Эта настройка действует только в тех случаях, когда сервер сам формирует такие блоки. Например, при INSERT-е через HTTP интерфейс, сервер парсит формат данных, и формирует блоки указанного размера. А при использовании clickhouse-client, клиент сам парсит данные, и настройка max\_insert\_block\_size на сервере не влияет на размер вставляемых блоков. From 7547b850d6b84a9e5ef5b83c31062821359e46c6 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 11 Aug 2020 17:14:06 +0300 Subject: [PATCH 252/374] performance comparison --- .../test/performance-comparison/download.sh | 9 ++++++--- .../test/performance-comparison/manual-run.sh | 20 +++++++++---------- 2 files changed, 16 insertions(+), 13 deletions(-) diff --git a/docker/test/performance-comparison/download.sh b/docker/test/performance-comparison/download.sh index 6cbfb77b290..ce953f59cd5 100755 --- a/docker/test/performance-comparison/download.sh +++ b/docker/test/performance-comparison/download.sh @@ -33,13 +33,16 @@ function download fi done - # Might have the same version on left and right (for testing). + # Might have the same version on left and right (for testing) -- in this case we just copy + # already downloaded 'right' to the 'left. There is the third case when we don't have to + # download anything, for example in some manual runs. In this case, SHAs are not set. if ! [ "$left_sha" = "$right_sha" ] then wget -nv -nd -c "$left_path" -O- | tar -C left --strip-components=1 -zxv & - else + elif [ "$right_sha" != "" ] + then mkdir left ||: - cp -a right/* left & + cp -an right/* left & fi for dataset_name in $datasets diff --git a/docker/test/performance-comparison/manual-run.sh b/docker/test/performance-comparison/manual-run.sh index 84a96223b0e..2cc40bf4648 100755 --- a/docker/test/performance-comparison/manual-run.sh +++ b/docker/test/performance-comparison/manual-run.sh @@ -15,24 +15,24 @@ function download mkdir left right db0 ||: "$script_dir/download.sh" ||: & - cp -vP "$repo_dir"/../build-gcc9-rel/programs/clickhouse* right & - cp -vP "$repo_dir"/../build-clang10-rel/programs/clickhouse* left & + cp -nvP "$repo_dir"/../build-gcc9-rel/programs/clickhouse* left & + cp -nvP "$repo_dir"/../build-clang10-rel/programs/clickhouse* right & wait } function configure { # Test files - cp -av "$repo_dir/tests/performance" right - cp -av "$repo_dir/tests/performance" left + cp -nav "$repo_dir/tests/performance" right + cp -nav "$repo_dir/tests/performance" left # Configs - cp -av "$script_dir/config" right - cp -av "$script_dir/config" left - cp -av "$repo_dir"/programs/server/config* right/config - cp -av "$repo_dir"/programs/server/user* right/config - cp -av "$repo_dir"/programs/server/config* left/config - cp -av "$repo_dir"/programs/server/user* left/config + cp -nav "$script_dir/config" right + cp -nav "$script_dir/config" left + cp -nav "$repo_dir"/programs/server/config* right/config + cp -nav "$repo_dir"/programs/server/user* right/config + cp -nav "$repo_dir"/programs/server/config* left/config + cp -nav "$repo_dir"/programs/server/user* left/config tree left } From 4036d9bd10b8223b2258999c8110c57b06388cb1 Mon Sep 17 00:00:00 2001 From: melin Date: Tue, 11 Aug 2020 22:35:28 +0800 Subject: [PATCH 253/374] antlr4 support complete create table grammar --- utils/grammar/ClickHouseLexer.g4 | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/utils/grammar/ClickHouseLexer.g4 b/utils/grammar/ClickHouseLexer.g4 index 766e30d2850..070fd042746 100644 --- a/utils/grammar/ClickHouseLexer.g4 +++ b/utils/grammar/ClickHouseLexer.g4 @@ -27,9 +27,11 @@ K_CHECK : C H E C K; K_CLUSTER : C L U S T E R; K_COLUMN : C O L U M N; K_COLLATE : C O L L A T E; +K_CODEC : C O D E C; K_CREATE : C R E A T E; K_CROSS : C R O S S; K_DAY : D A Y; +K_DELETE : D E L E T E; K_DESCRIBE : D E S C R I B E; K_DESCENDING : D E S C E N D I N G; K_DESC : D E S C; @@ -37,6 +39,7 @@ K_DATABASE : D A T A B A S E; K_DATABASES : D A T A B A S E S; K_DEFAULT : D E F A U L T; K_DETACH : D E T A C H; +K_DISK : D I S K; K_DISTINCT : D I S T I N C T; K_DROP : D R O P; K_ELSE : E L S E; @@ -104,8 +107,10 @@ K_TEST : T E S T; K_THEN : T H E N; K_TOTALS : T O T A L S; K_TO : T O; +K_TTL : T T L; K_OUTER: O U T E R; K_VALUES : V A L U E S; +K_VOLUME : V O L U M E; K_VIEW : V I E W; K_UNION : U N I O N; K_USE : U S E; From d1a770d863203e3a84011bfde84ff8bb9eace0ec Mon Sep 17 00:00:00 2001 From: melin Date: Tue, 11 Aug 2020 22:36:19 +0800 Subject: [PATCH 254/374] antlr4 support complete create table grammar --- utils/grammar/ClickHouseParser.g4 | 25 +++++++++++++++++-------- 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/utils/grammar/ClickHouseParser.g4 b/utils/grammar/ClickHouseParser.g4 index c2b4a5f6074..9ecf4fc93e4 100644 --- a/utils/grammar/ClickHouseParser.g4 +++ b/utils/grammar/ClickHouseParser.g4 @@ -49,7 +49,7 @@ select_query_main select_groupby_step? select_having_step? select_orderby_step? select_limitby_step? select_limit_step? - select_settings_step? + settings_step? ; select_with_step @@ -122,7 +122,7 @@ select_limitby_step : K_LIMIT NUMERIC_LITERAL K_BY not_empty_expression_list ; -select_settings_step +settings_step : K_SETTINGS assignment_list ; @@ -143,7 +143,12 @@ create_query : ( K_CREATE | K_ATTACH ) K_TEMPORARY? ( K_DATABASE ( K_IF K_NOT K_EXISTS ) ? database_name | K_TABLE ( K_IF K_NOT K_EXISTS ) ? full_table_name ( K_ON K_CLUSTER cluster_name ) ? - ( LPAREN column_declaration_list RPAREN engine ( K_AS select_query ) ? // если VIEW - то есть и колонки и select. + ( LPAREN column_declaration_list RPAREN engine ( K_AS select_query ) ? + K_ORDER K_BY LPAREN order_by_expression_list RPAREN + (K_PARTITION K_BY partitionExpr=expr)? + (K_SAMPLE K_BY sampleExpr=expr)? + table_ttl_list? + settings_step? | engine K_AS ( select_query | full_table_name engine? // wtf ) @@ -315,6 +320,14 @@ order_by_element : expression_with_optional_alias ( K_DESC | K_DESCENDING | K_ASC | K_ASCENDING ) ? ( K_NULLS ( K_FIRST | K_LAST ) ) ? ( K_COLLATE STRING_LITERAL ) ? ; +table_ttl_list + : K_TTL table_ttl_declaration ( COMMA table_ttl_declaration ) * + ; + +table_ttl_declaration + : ttlExpr=expr (K_DELETE | K_TO K_DISK diskVal=STRING_LITERAL | K_TO K_VOLUME volumeVal=STRING_LITERAL)? + ; + nested_table : identifier LPAREN name_type_pair_list RPAREN ; @@ -336,10 +349,7 @@ column_declaration_list ; column_declaration - : column_name - ( ( K_DEFAULT | K_MATERIALIZED | K_ALIAS ) expr - | column_type - ) + : column_name column_type (( K_DEFAULT | K_MATERIALIZED | K_ALIAS ) expr)? (K_CODEC(expr))? (K_TTL ttlExpr=expr)? ; column_name @@ -581,4 +591,3 @@ err throw new RuntimeException("UNEXPECTED_CHAR=" + $UNEXPECTED_CHAR.text); } ; - From 4bf3a3e704d5ac3d2a6135953bdfc1e1fe97c5c5 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 11 Aug 2020 18:28:42 +0300 Subject: [PATCH 255/374] DOCSUP-711: Add description for setting output_format_json_quote_denormals (#13391) * DOCSUP-711: (en) In settings added description for output_format_json_quote_denormals. * DOCSUP-711: Fix invalid anchor. * DOCSUP-711: (ru) In settings added description for output_format_json_quote_denormals. * DOCSUP-711: Change values styles. * DOCSUP-711: Add a link from the setting description to the JSON format description. (by pr comment) Add a link from the JSON format description to the setting description. (by pr comment) * DOCSUP-711: Rewrite sentence. * DOCSUP-711: (ru) Added link from settings.md to JSON format. * DOCSUP-711: (en) Added example of output with the setting (by pr comment). * DOCSUP-711: Add example of output with the setting. * DOCSUP-711: Minor fix. * DOCSUP-711: Update example of settings using. * DOCSUP-711: Minor fix. * Update docs/en/interfaces/formats.md by pr Co-authored-by: BayoNet * [wip]Update docs/en/operations/settings/settings.md Co-authored-by: BayoNet * DOCSUP-711: (en) Add sample table and fix pr comments. * DOCSUP-711: (ru) Updated from en and by pr comments. * DOCSUP-711: (ru) Updated from en and by pr comments. * DOCSUP-711: Added name for sample table. * Update docs/ru/operations/settings/settings.md by pr comment Co-authored-by: BayoNet * Update docs/ru/interfaces/formats.md by pr comment Co-authored-by: BayoNet * Update settings.md * Update settings.md Co-authored-by: romanzhukov Co-authored-by: BayoNet --- docs/en/interfaces/formats.md | 2 +- docs/en/operations/settings/settings.md | 99 +++++++++++++++++++++++++ docs/ru/interfaces/formats.md | 2 +- docs/ru/operations/settings/settings.md | 99 +++++++++++++++++++++++++ 4 files changed, 200 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index a6b424835f7..9d3965b4a9c 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -464,7 +464,7 @@ If the query contains GROUP BY, rows\_before\_limit\_at\_least is the exact numb This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). -ClickHouse supports [NULL](../sql-reference/syntax.md), which is displayed as `null` in the JSON output. +ClickHouse supports [NULL](../sql-reference/syntax.md), which is displayed as `null` in the JSON output. To enable `+nan`, `-nan`, `+inf`, `-inf` values in output, set the [output\_format\_json\_quote\_denormals](../operations/settings/settings.md#settings-output_format_json_quote_denormals) to 1. See also the [JSONEachRow](#jsoneachrow) format. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 27d5a2e6f79..7f130b75a64 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -998,6 +998,105 @@ The results of the compilation are saved in the build directory in the form of . If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes. +## output\_format\_json\_quote\_denormals {#settings-output_format_json_quote_denormals} + +Enables `+nan`, `-nan`, `+inf`, `-inf` outputs in [JSON](../../interfaces/formats.md#json) output format. + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: 0. + +**Example** + +Consider the following table `account_orders`: + +```text +┌─id─┬─name───┬─duration─┬─period─┬─area─┐ +│ 1 │ Andrew │ 20 │ 0 │ 400 │ +│ 2 │ John │ 40 │ 0 │ 0 │ +│ 3 │ Bob │ 15 │ 0 │ -100 │ +└────┴────────┴──────────┴────────┴──────┘ +``` + +When `output_format_json_quote_denormals = 0`, the query returns `null` values in output: + +```sql +SELECT area/period FROM account_orders FORMAT JSON; +``` + +```json +{ + "meta": + [ + { + "name": "divide(area, period)", + "type": "Float64" + } + ], + + "data": + [ + { + "divide(area, period)": null + }, + { + "divide(area, period)": null + }, + { + "divide(area, period)": null + } + ], + + "rows": 3, + + "statistics": + { + "elapsed": 0.003648093, + "rows_read": 3, + "bytes_read": 24 + } +} +``` + +When `output_format_json_quote_denormals = 1`, the query returns: + +```json +{ + "meta": + [ + { + "name": "divide(area, period)", + "type": "Float64" + } + ], + + "data": + [ + { + "divide(area, period)": "inf" + }, + { + "divide(area, period)": "-nan" + }, + { + "divide(area, period)": "-inf" + } + ], + + "rows": 3, + + "statistics": + { + "elapsed": 0.000070241, + "rows_read": 3, + "bytes_read": 24 + } +} +``` + ## format\_csv\_delimiter {#settings-format_csv_delimiter} The character interpreted as a delimiter in the CSV data. By default, the delimiter is `,`. diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 6e7bb796bb8..054f75e8da8 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -432,7 +432,7 @@ JSON совместим с JavaScript. Для этого, дополнитель Этот формат подходит только для вывода результата выполнения запроса, но не для парсинга (приёма данных для вставки в таблицу). -ClickHouse поддерживает [NULL](../sql-reference/syntax.md), который при выводе JSON будет отображен как `null`. +ClickHouse поддерживает [NULL](../sql-reference/syntax.md), который при выводе JSON будет отображен как `null`. Чтобы включить отображение в результате значений `+nan`, `-nan`, `+inf`, `-inf`, установите параметр [output\_format\_json\_quote\_denormals](../operations/settings/settings.md#settings-output_format_json_quote_denormals) равным 1. Смотрите также формат [JSONEachRow](#jsoneachrow) . diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 589b88597c8..2fce04b4c69 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -908,6 +908,105 @@ load_balancing = first_or_random Если значение истинно, то при использовании JSON\* форматов UInt64 и Int64 числа выводятся в кавычках (из соображений совместимости с большинством реализаций JavaScript), иначе - без кавычек. +## output\_format\_json\_quote\_denormals {#settings-output_format_json_quote_denormals} + +При выводе данных в формате [JSON](../../interfaces/formats.md#json) включает отображение значений `+nan`, `-nan`, `+inf`, `-inf`. + +Возможные значения: + +- 0 — выключена. +- 1 — включена. + +Значение по умолчанию: 0. + +**Пример** + +Рассмотрим следующую таблицу `account_orders`: + +```text +┌─id─┬─name───┬─duration─┬─period─┬─area─┐ +│ 1 │ Andrew │ 20 │ 0 │ 400 │ +│ 2 │ John │ 40 │ 0 │ 0 │ +│ 3 │ Bob │ 15 │ 0 │ -100 │ +└────┴────────┴──────────┴────────┴──────┘ +``` + +Когда `output_format_json_quote_denormals = 0`, следующий запрос возвращает значения `null`. + +```sql +SELECT area/period FROM account_orders FORMAT JSON; +``` + +```json +{ + "meta": + [ + { + "name": "divide(area, period)", + "type": "Float64" + } + ], + + "data": + [ + { + "divide(area, period)": null + }, + { + "divide(area, period)": null + }, + { + "divide(area, period)": null + } + ], + + "rows": 3, + + "statistics": + { + "elapsed": 0.003648093, + "rows_read": 3, + "bytes_read": 24 + } +} +``` + +Если `output_format_json_quote_denormals = 1`, то запрос вернет: + +```json +{ + "meta": + [ + { + "name": "divide(area, period)", + "type": "Float64" + } + ], + + "data": + [ + { + "divide(area, period)": "inf" + }, + { + "divide(area, period)": "-nan" + }, + { + "divide(area, period)": "-inf" + } + ], + + "rows": 3, + + "statistics": + { + "elapsed": 0.000070241, + "rows_read": 3, + "bytes_read": 24 + } +} +``` + ## format\_csv\_delimiter {#settings-format_csv_delimiter} Символ, интерпретируемый как разделитель в данных формата CSV. По умолчанию — `,`. From f9ef5e2833a89de7cb312eec04c18428e284b287 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 11 Aug 2020 19:23:06 +0300 Subject: [PATCH 256/374] Fixed result order in hash set test This is needed to fix the test in 20.6, where apparently the order is different. --- tests/queries/0_stateless/01428_hash_set_nan_key.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01428_hash_set_nan_key.sql b/tests/queries/0_stateless/01428_hash_set_nan_key.sql index 0408e30070a..f9875f9ed99 100644 --- a/tests/queries/0_stateless/01428_hash_set_nan_key.sql +++ b/tests/queries/0_stateless/01428_hash_set_nan_key.sql @@ -5,6 +5,6 @@ SELECT DISTINCT number % inf FROM numbers(1000); SELECT topKWeightedMerge(1)(initializeAggregation('topKWeightedState(1)', nan, arrayJoin(range(10)))); -select number % inf k from numbers(256) group by k; +select number % inf k from numbers(256) group by k order by k; -SELECT uniqExact(reinterpretAsFloat64(reinterpretAsFixedString(reinterpretAsUInt64(reinterpretAsFixedString(nan)) + number))) FROM numbers(10); +SELECT uniqExact(reinterpretAsFloat64(reinterpretAsFixedString(reinterpretAsUInt64(reinterpretAsFixedString(nan)) + number))) n FROM numbers(10) order by n; From c5c41ce90d1d1ca2f71a325ac00e34ec63b555db Mon Sep 17 00:00:00 2001 From: Yuriy Korzhenevskiy Date: Tue, 11 Aug 2020 21:29:37 +0300 Subject: [PATCH 257/374] Fix typos and remove bash sign for copy-and-go docs command. --- docs/ru/getting-started/install.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index 1cef192b518..d6afbc8d52c 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -7,7 +7,7 @@ ClickHouse может работать на любой операционной Предварительно собранные пакеты компилируются для x86\_64 и используют набор инструкций SSE 4.2, поэтому, если не указано иное, его поддержка в используемом процессоре, становится дополнительным требованием к системе. Вот команда, чтобы проверить, поддерживает ли текущий процессор SSE 4.2: ``` bash -$ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" +grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" ``` Чтобы запустить ClickHouse на процессорах, которые не поддерживают SSE 4.2, либо имеют архитектуру AArch64 или PowerPC64LE, необходимо самостоятельно [собрать ClickHouse из исходного кода](#from-sources) с соответствующими настройками конфигурации. @@ -84,7 +84,7 @@ sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh ### Из исполняемых файлов для нестандартных окружений {#from-binaries-non-linux} -Для других операционных систем и арихитектуры AArch64, сборки ClickHouse предоставляются в виде кросс-компилированного бинарника с последнего коммита ветки master (с задержкой в несколько часов). +Для других операционных систем и архитектуры AArch64, сборки ClickHouse предоставляются в виде кросс-компилированного бинарника с последнего коммита ветки master (с задержкой в несколько часов). - [macOS](https://builds.clickhouse.tech/master/macos/clickhouse) — `curl -O 'https://builds.clickhouse.tech/master/macos/clickhouse' && chmod a+x ./clickhouse` - [AArch64](https://builds.clickhouse.tech/master/aarch64/clickhouse) — `curl -O 'https://builds.clickhouse.tech/master/aarch64/clickhouse' && chmod a+x ./clickhouse` @@ -115,7 +115,7 @@ sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh Для запуска сервера в качестве демона, выполните: ``` bash -$ sudo service clickhouse-server start +sudo service clickhouse-server start ``` Смотрите логи в директории `/var/log/clickhouse-server/`. @@ -125,7 +125,7 @@ $ sudo service clickhouse-server start Также можно запустить сервер вручную из консоли: ``` bash -$ clickhouse-server --config-file=/etc/clickhouse-server/config.xml +clickhouse-server --config-file=/etc/clickhouse-server/config.xml ``` При этом, лог будет выводиться в консоль, что удобно для разработки. @@ -134,7 +134,7 @@ $ clickhouse-server --config-file=/etc/clickhouse-server/config.xml После запуска сервера, соединиться с ним можно с помощью клиента командной строки: ``` bash -$ clickhouse-client +clickhouse-client ``` По умолчанию он соединяется с localhost:9000, от имени пользователя `default` без пароля. Также клиент может быть использован для соединения с удалённым сервером с помощью аргумента `--host`. From 33197fee5bb1c4b02f693f27552cc3541613f5c5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 11 Aug 2020 22:03:04 +0300 Subject: [PATCH 258/374] Build ClickHouse with fresh tzdata --- docker/packager/binary/build.sh | 3 +++ docker/packager/deb/build.sh | 3 +++ 2 files changed, 6 insertions(+) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 72adba5d762..07b67d0db9a 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -2,6 +2,9 @@ set -x -e +# Update tzdata to the latest version. It is embedded into clickhouse binary. +sudo apt-get update && sudo apt-get install tzdata + mkdir -p build/cmake/toolchain/darwin-x86_64 tar xJf MacOSX10.14.sdk.tar.xz -C build/cmake/toolchain/darwin-x86_64 --strip-components=1 diff --git a/docker/packager/deb/build.sh b/docker/packager/deb/build.sh index fbaa0151c6b..8b26bbb19cb 100755 --- a/docker/packager/deb/build.sh +++ b/docker/packager/deb/build.sh @@ -2,6 +2,9 @@ set -x -e +# Update tzdata to the latest version. It is embedded into clickhouse binary. +sudo apt-get update && sudo apt-get install tzdata + ccache --show-stats ||: ccache --zero-stats ||: build/release --no-pbuilder $ALIEN_PKGS | ts '%Y-%m-%d %H:%M:%S' From ea46ee74ae4b55f059b833d74a4c29d50b8a32b8 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 11 Aug 2020 22:08:32 +0300 Subject: [PATCH 259/374] Don't allocate Executor instance on each disk->getExecutor() call. --- src/Disks/Executor.h | 25 +++++++++++++++++++++++++ src/Disks/IDisk.cpp | 34 ++-------------------------------- src/Disks/IDisk.h | 9 +++++++-- src/Disks/S3/DiskS3.cpp | 8 ++------ src/Disks/S3/DiskS3.h | 2 -- 5 files changed, 36 insertions(+), 42 deletions(-) diff --git a/src/Disks/Executor.h b/src/Disks/Executor.h index 61afb569903..7330bcdd559 100644 --- a/src/Disks/Executor.h +++ b/src/Disks/Executor.h @@ -14,4 +14,29 @@ public: virtual std::future execute(std::function task) = 0; }; +/// Executes task synchronously in case when disk doesn't support async operations. +class SyncExecutor : public Executor +{ +public: + SyncExecutor() = default; + std::future execute(std::function task) override + { + auto promise = std::make_shared>(); + try + { + task(); + promise->set_value(); + } + catch (...) + { + try + { + promise->set_exception(std::current_exception()); + } + catch (...) { } + } + return promise->get_future(); + } +}; + } diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index fd46f693c31..5ae96ca6c23 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -59,10 +59,10 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p void IDisk::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) { - auto exec = to_disk->getExecutor(); + auto & exec = to_disk->getExecutor(); ResultsCollector results; - asyncCopy(*this, from_path, *to_disk, to_path, *exec, results); + asyncCopy(*this, from_path, *to_disk, to_path, exec, results); for (auto & result : results) result.wait(); @@ -70,36 +70,6 @@ void IDisk::copy(const String & from_path, const std::shared_ptr & to_dis result.get(); } -/// Executes task synchronously in case when disk doesn't support async operations. -class SyncExecutor : public Executor -{ -public: - SyncExecutor() = default; - std::future execute(std::function task) override - { - auto promise = std::make_shared>(); - try - { - task(); - promise->set_value(); - } - catch (...) - { - try - { - promise->set_exception(std::current_exception()); - } - catch (...) { } - } - return promise->get_future(); - } -}; - -std::unique_ptr IDisk::getExecutor() -{ - return std::make_unique(); -} - void IDisk::truncateFile(const String &, size_t) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate operation is not implemented for disk of type {}", getType()); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index f59c8551633..53dc4999dc4 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -29,7 +30,6 @@ using Reservations = std::vector; class ReadBufferFromFileBase; class WriteBufferFromFileBase; -class Executor; /** * Mode of opening a file for write. @@ -67,6 +67,9 @@ using SpacePtr = std::shared_ptr; class IDisk : public Space { public: + /// Default constructor. + explicit IDisk(std::unique_ptr executor_ = std::make_unique()) : executor(std::move(executor_)) { } + /// Root path for all files stored on the disk. /// It's not required to be a local filesystem path. virtual const String & getPath() const = 0; @@ -182,7 +185,9 @@ public: private: /// Returns executor to perform asynchronous operations. - virtual std::unique_ptr getExecutor(); + Executor & getExecutor() { return *executor; } + + std::unique_ptr executor; }; using DiskPtr = std::shared_ptr; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 06f0fec4e86..5aa57518c83 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -483,7 +483,8 @@ DiskS3::DiskS3( size_t min_upload_part_size_, size_t min_multi_part_upload_size_, size_t min_bytes_for_seek_) - : name(std::move(name_)) + : IDisk(std::make_unique()) + , name(std::move(name_)) , client(std::move(client_)) , proxy_configuration(std::move(proxy_configuration_)) , bucket(std::move(bucket_)) @@ -745,9 +746,4 @@ void DiskS3::setReadOnly(const String & path) Poco::File(metadata_path + path).setReadOnly(true); } -std::unique_ptr DiskS3::getExecutor() -{ - return std::make_unique(); -} - } diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index ac576666f2e..34f00af6439 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -105,8 +105,6 @@ public: private: bool tryReserve(UInt64 bytes); - std::unique_ptr getExecutor() override; - private: const String name; std::shared_ptr client; From f950f870e5d9b569e79ae8a684a2f02fe9750567 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 12 Aug 2020 00:19:41 +0400 Subject: [PATCH 260/374] changelog for 20.5 (#13422) * Update CHANGELOG.md * Update CHANGELOG.md * better * date --- CHANGELOG.md | 43 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5850c70043d..2a3249c2701 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -136,6 +136,49 @@ ## ClickHouse release 20.5 +### ClickHouse release v20.5.4.40-stable 2020-08-10 + +#### Bug Fix + +* Fixed wrong index analysis with functions. It could lead to pruning wrong parts, while reading from `MergeTree` tables. Fixes [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed performance with large tuples, which are interpreted as functions in `IN` section. The case when user write `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed memory tracking for input_format_parallel_parsing (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). +* Fixed bloom filter index with const expression. This fixes [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572). [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `SIGSEGV` in `StorageKafka` when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). +* Added support for function `if` with `Array(UUID)` arguments. This fixes [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed lack of aliases with function `any`. [#12593](https://github.com/ClickHouse/ClickHouse/pull/12593) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). +* Remove data for Distributed tables (blocks from async INSERTs) on DROP TABLE. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). +* Fixed bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). +* Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed race condition in live view tables which could cause data duplication. [#12519](https://github.com/ClickHouse/ClickHouse/pull/12519) ([vzakaznikov](https://github.com/vzakaznikov)). +* Fixed performance issue, while reading from compact parts. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed backwards compatibility in binary format of `AggregateFunction(avg, ...)` values. This fixes [#12342](https://github.com/ClickHouse/ClickHouse/issues/12342). [#12486](https://github.com/ClickHouse/ClickHouse/pull/12486) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the deadlock if `text_log` is enabled. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed overflow when very large LIMIT or OFFSET is specified. This fixes [#10470](https://github.com/ClickHouse/ClickHouse/issues/10470). This fixes [#11372](https://github.com/ClickHouse/ClickHouse/issues/11372). [#12427](https://github.com/ClickHouse/ClickHouse/pull/12427) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed possible segfault if StorageMerge. Closes [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). +* Reverts change introduced in [#11079](https://github.com/ClickHouse/ClickHouse/issues/11079) to resolve [#12098](https://github.com/ClickHouse/ClickHouse/issues/12098). [#12397](https://github.com/ClickHouse/ClickHouse/pull/12397) ([Mike](https://github.com/myrrc)). +* Avoid exception when negative or floating point constant is used in WHERE condition for indexed tables. This fixes [#11905](https://github.com/ClickHouse/ClickHouse/issues/11905). [#12384](https://github.com/ClickHouse/ClickHouse/pull/12384) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow to CLEAR column even if there are depending DEFAULT expressions. This fixes [#12333](https://github.com/ClickHouse/ClickHouse/issues/12333). [#12378](https://github.com/ClickHouse/ClickHouse/pull/12378) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed TOTALS/ROLLUP/CUBE for aggregate functions with `-State` and `Nullable` arguments. This fixes [#12163](https://github.com/ClickHouse/ClickHouse/issues/12163). [#12376](https://github.com/ClickHouse/ClickHouse/pull/12376) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed SIGSEGV if there is an message with error in the middle of the batch in `Kafka` Engine. [#12302](https://github.com/ClickHouse/ClickHouse/pull/12302) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the behaviour when `SummingMergeTree` engine sums up columns from partition key. Added an exception in case of explicit definition of columns to sum which intersects with partition key columns. This fixes [#7867](https://github.com/ClickHouse/ClickHouse/issues/7867). [#12173](https://github.com/ClickHouse/ClickHouse/pull/12173) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed transform of query to send to external DBMS (e.g. MySQL, ODBC) in presense of aliases. This fixes [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed bug which leads to incorrect table metadata in ZooKeepeer for ReplicatedVersionedCollapsingMergeTree tables. Fixes [#12093](https://github.com/ClickHouse/ClickHouse/issues/12093). [#12121](https://github.com/ClickHouse/ClickHouse/pull/12121) ([alesapin](https://github.com/alesapin)). +* Fixed unnecessary limiting the number of threads for selects from `VIEW`. Fixes [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed crash in JOIN with LowCardinality type with `join_algorithm=partial_merge`. [#12035](https://github.com/ClickHouse/ClickHouse/pull/12035) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixed wrong result for `if()` with NULLs in condition. [#11807](https://github.com/ClickHouse/ClickHouse/pull/11807) ([Artem Zuikov](https://github.com/4ertus2)). + +#### Performance Improvement + +* Index not used for IN operator with literals", performance regression introduced around v19.3. This fixes "[#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). + +#### Build/Testing/Packaging Improvement + +* Install `ca-certificates` before the first `apt-get update` in Dockerfile. [#12095](https://github.com/ClickHouse/ClickHouse/pull/12095) ([Ivan Blinkov](https://github.com/blinkov)). + + ### ClickHouse release v20.5.2.7-stable 2020-07-02 #### Backward Incompatible Change From 62c43784691f5a56be3fe8a1e1d7e6f99d132358 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 12 Aug 2020 00:19:56 +0400 Subject: [PATCH 261/374] changelog for 20.4 (#13522) * Update CHANGELOG.md * better --- CHANGELOG.md | 74 ++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 74 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2a3249c2701..2bcb57c0dde 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -534,6 +534,80 @@ ## ClickHouse release v20.4 +### ClickHouse release v20.4.8.99-stable 2020-08-10 + +#### Bug Fix + +* Fixed error in `parseDateTimeBestEffort` function when unix timestamp was passed as an argument. This fixes [#13362](https://github.com/ClickHouse/ClickHouse/issues/13362). [#13441](https://github.com/ClickHouse/ClickHouse/pull/13441) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potentially low performance and slightly incorrect result for `uniqExact`, `topK`, `sumDistinct` and similar aggregate functions called on Float types with NaN values. It also triggered assert in debug build. This fixes [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed function if with nullable constexpr as cond that is not literal NULL. Fixes [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed assert in `arrayElement` function in case of array elements are Nullable and array subscript is also Nullable. This fixes [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed wrong index analysis with functions. It could lead to pruning wrong parts, while reading from `MergeTree` tables. Fixes [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible extra overflow row in data which could appear for queries `WITH TOTALS`. [#12747](https://github.com/ClickHouse/ClickHouse/pull/12747) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed performance with large tuples, which are interpreted as functions in `IN` section. The case when user write `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed memory tracking for `input_format_parallel_parsing` (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). +* Fixed [#12293](https://github.com/ClickHouse/ClickHouse/issues/12293) allow push predicate when subquery contains with clause. [#12663](https://github.com/ClickHouse/ClickHouse/pull/12663) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572) fix bloom filter index with const expression. [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `SIGSEGV` in `StorageKafka` when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). +* Added support for function `if` with `Array(UUID)` arguments. This fixes [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). +* Removed data for Distributed tables (blocks from async INSERTs) on DROP TABLE. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). +* Fixed bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). +* Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed performance issue, while reading from compact parts. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed crash in JOIN with dictionary when we are joining over expression of dictionary key: `t JOIN dict ON expr(dict.id) = t.id`. Disable dictionary join optimisation for this case. [#12458](https://github.com/ClickHouse/ClickHouse/pull/12458) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixed possible segfault if StorageMerge. Closes [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). +* Fixed order of columns in `WITH FILL` modifier. Previously order of columns of `ORDER BY` statement wasn't respected. [#12306](https://github.com/ClickHouse/ClickHouse/pull/12306) ([Anton Popov](https://github.com/CurtizJ)). +* Avoid "bad cast" exception when there is an expression that filters data by virtual columns (like `_table` in `Merge` tables) or by "index" columns in system tables such as filtering by database name when querying from `system.tables`, and this expression returns `Nullable` type. This fixes [#12166](https://github.com/ClickHouse/ClickHouse/issues/12166). [#12305](https://github.com/ClickHouse/ClickHouse/pull/12305) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Show error after TrieDictionary failed to load. [#12290](https://github.com/ClickHouse/ClickHouse/pull/12290) ([Vitaly Baranov](https://github.com/vitlibar)). +* The function `arrayFill` worked incorrectly for empty arrays that may lead to crash. This fixes [#12263](https://github.com/ClickHouse/ClickHouse/issues/12263). [#12279](https://github.com/ClickHouse/ClickHouse/pull/12279) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Implemented conversions to the common type for `LowCardinality` types. This allows to execute UNION ALL of tables with columns of LowCardinality and other columns. This fixes [#8212](https://github.com/ClickHouse/ClickHouse/issues/8212). This fixes [#4342](https://github.com/ClickHouse/ClickHouse/issues/4342). [#12275](https://github.com/ClickHouse/ClickHouse/pull/12275) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the behaviour when during multiple sequential inserts in `StorageFile` header for some special types was written more than once. This fixed [#6155](https://github.com/ClickHouse/ClickHouse/issues/6155). [#12197](https://github.com/ClickHouse/ClickHouse/pull/12197) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed logical functions for UInt8 values when they are not equal to 0 or 1. [#12196](https://github.com/ClickHouse/ClickHouse/pull/12196) ([Alexander Kazakov](https://github.com/Akazz)). +* Cap max_memory_usage* limits to the process resident memory. [#12182](https://github.com/ClickHouse/ClickHouse/pull/12182) ([Azat Khuzhin](https://github.com/azat)). +* Fixed `dictGet` arguments check during GROUP BY injective functions elimination. [#12179](https://github.com/ClickHouse/ClickHouse/pull/12179) ([Azat Khuzhin](https://github.com/azat)). +* Don't split the dictionary source's table name into schema and table name itself if ODBC connection doesn't support schema. [#12165](https://github.com/ClickHouse/ClickHouse/pull/12165) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed wrong logic in `ALTER DELETE` that leads to deleting of records when condition evaluates to NULL. This fixes [#9088](https://github.com/ClickHouse/ClickHouse/issues/9088). This closes [#12106](https://github.com/ClickHouse/ClickHouse/issues/12106). [#12153](https://github.com/ClickHouse/ClickHouse/pull/12153) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed transform of query to send to external DBMS (e.g. MySQL, ODBC) in presense of aliases. This fixes [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential overflow in integer division. This fixes [#12119](https://github.com/ClickHouse/ClickHouse/issues/12119). [#12140](https://github.com/ClickHouse/ClickHouse/pull/12140) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential infinite loop in `greatCircleDistance`, `geoDistance`. This fixes [#12117](https://github.com/ClickHouse/ClickHouse/issues/12117). [#12137](https://github.com/ClickHouse/ClickHouse/pull/12137) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Normalize "pid" file handling. In previous versions the server may refuse to start if it was killed without proper shutdown and if there is another process that has the same pid as previously runned server. Also pid file may be removed in unsuccessful server startup even if there is another server running. This fixes [#3501](https://github.com/ClickHouse/ClickHouse/issues/3501). [#12133](https://github.com/ClickHouse/ClickHouse/pull/12133) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed handling dependency of table with ENGINE=Dictionary on dictionary. This fixes [#10994](https://github.com/ClickHouse/ClickHouse/issues/10994). This fixes [#10397](https://github.com/ClickHouse/ClickHouse/issues/10397). [#12116](https://github.com/ClickHouse/ClickHouse/pull/12116) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed performance for selects with `UNION` caused by wrong limit for the total number of threads. Fixes [#12030](https://github.com/ClickHouse/ClickHouse/issues/12030). [#12103](https://github.com/ClickHouse/ClickHouse/pull/12103) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed segfault with `-StateResample` combinators. [#12092](https://github.com/ClickHouse/ClickHouse/pull/12092) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed empty `result_rows` and `result_bytes` metrics in `system.quey_log` for selects. Fixes [#11595](https://github.com/ClickHouse/ClickHouse/issues/11595). [#12089](https://github.com/ClickHouse/ClickHouse/pull/12089) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed unnecessary limiting the number of threads for selects from `VIEW`. Fixes [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible crash while using wrong type for `PREWHERE`. Fixes [#12053](https://github.com/ClickHouse/ClickHouse/issues/12053), [#12060](https://github.com/ClickHouse/ClickHouse/issues/12060). [#12060](https://github.com/ClickHouse/ClickHouse/pull/12060) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed error `Expected single dictionary argument for function` for function `defaultValueOfArgumentType` with `LowCardinality` type. Fixes [#11808](https://github.com/ClickHouse/ClickHouse/issues/11808). [#12056](https://github.com/ClickHouse/ClickHouse/pull/12056) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed error `Cannot capture column` for higher-order functions with `Tuple(LowCardinality)` argument. Fixes [#9766](https://github.com/ClickHouse/ClickHouse/issues/9766). [#12055](https://github.com/ClickHouse/ClickHouse/pull/12055) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Parse tables metadata in parallel when loading database. This fixes slow server startup when there are large number of tables. [#12045](https://github.com/ClickHouse/ClickHouse/pull/12045) ([tavplubix](https://github.com/tavplubix)). +* Make `topK` aggregate function return Enum for Enum types. This fixes [#3740](https://github.com/ClickHouse/ClickHouse/issues/3740). [#12043](https://github.com/ClickHouse/ClickHouse/pull/12043) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed constraints check if constraint is a constant expression. This fixes [#11360](https://github.com/ClickHouse/ClickHouse/issues/11360). [#12042](https://github.com/ClickHouse/ClickHouse/pull/12042) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed incorrect comparison of tuples with `Nullable` columns. Fixes [#11985](https://github.com/ClickHouse/ClickHouse/issues/11985). [#12039](https://github.com/ClickHouse/ClickHouse/pull/12039) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed calculation of access rights when allow_introspection_functions=0. [#12031](https://github.com/ClickHouse/ClickHouse/pull/12031) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed wrong result and potential crash when invoking function `if` with arguments of type `FixedString` with different sizes. This fixes [#11362](https://github.com/ClickHouse/ClickHouse/issues/11362). [#12021](https://github.com/ClickHouse/ClickHouse/pull/12021) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* A query with function `neighbor` as the only returned expression may return empty result if the function is called with offset `-9223372036854775808`. This fixes [#11367](https://github.com/ClickHouse/ClickHouse/issues/11367). [#12019](https://github.com/ClickHouse/ClickHouse/pull/12019) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed calculation of access rights when allow_ddl=0. [#12015](https://github.com/ClickHouse/ClickHouse/pull/12015) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed potential array size overflow in generateRandom that may lead to crash. This fixes [#11371](https://github.com/ClickHouse/ClickHouse/issues/11371). [#12013](https://github.com/ClickHouse/ClickHouse/pull/12013) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential floating point exception. This closes [#11378](https://github.com/ClickHouse/ClickHouse/issues/11378). [#12005](https://github.com/ClickHouse/ClickHouse/pull/12005) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed wrong setting name in log message at server startup. [#11997](https://github.com/ClickHouse/ClickHouse/pull/11997) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `Query parameter was not set` in `Values` format. Fixes [#11918](https://github.com/ClickHouse/ClickHouse/issues/11918). [#11936](https://github.com/ClickHouse/ClickHouse/pull/11936) ([tavplubix](https://github.com/tavplubix)). +* Keep aliases for substitutions in query (parametrized queries). This fixes [#11914](https://github.com/ClickHouse/ClickHouse/issues/11914). [#11916](https://github.com/ClickHouse/ClickHouse/pull/11916) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed bug with no moves when changing storage policy from default one. [#11893](https://github.com/ClickHouse/ClickHouse/pull/11893) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fixed potential floating point exception when parsing `DateTime64`. This fixes [#11374](https://github.com/ClickHouse/ClickHouse/issues/11374). [#11875](https://github.com/ClickHouse/ClickHouse/pull/11875) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed memory accounting via HTTP interface (can be significant with `wait_end_of_query=1`). [#11840](https://github.com/ClickHouse/ClickHouse/pull/11840) ([Azat Khuzhin](https://github.com/azat)). +* Parse metadata stored in zookeeper before checking for equality. [#11739](https://github.com/ClickHouse/ClickHouse/pull/11739) ([Azat Khuzhin](https://github.com/azat)). + +#### Performance Improvement + +* Index not used for IN operator with literals", performance regression introduced around v19.3. This fixes "[#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). + +#### Build/Testing/Packaging Improvement + +* Install `ca-certificates` before the first `apt-get update` in Dockerfile. [#12095](https://github.com/ClickHouse/ClickHouse/pull/12095) ([Ivan Blinkov](https://github.com/blinkov)). + + ### ClickHouse release v20.4.6.53-stable 2020-06-25 #### Bug Fix From 0bb61a48ec2f60fe8fa8634caf9c00b3fcfc8e5a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 12 Aug 2020 00:20:10 +0400 Subject: [PATCH 262/374] changelog for 20.3 (#13523) * Update CHANGELOG.md * better --- CHANGELOG.md | 64 ++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 64 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2bcb57c0dde..927bba1198f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1015,6 +1015,70 @@ No changes compared to v20.4.3.16-stable. ## ClickHouse release v20.3 +### ClickHouse release v20.3.16.165-lts 2020-08-10 + +#### Bug Fix + +* Fixed error in `parseDateTimeBestEffort` function when unix timestamp was passed as an argument. This fixes [#13362](https://github.com/ClickHouse/ClickHouse/issues/13362). [#13441](https://github.com/ClickHouse/ClickHouse/pull/13441) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potentially low performance and slightly incorrect result for `uniqExact`, `topK`, `sumDistinct` and similar aggregate functions called on Float types with `NaN` values. It also triggered assert in debug build. This fixes [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed function if with nullable constexpr as cond that is not literal NULL. Fixes [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed assert in `arrayElement` function in case of array elements are Nullable and array subscript is also Nullable. This fixes [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible extra overflow row in data which could appear for queries `WITH TOTALS`. [#12747](https://github.com/ClickHouse/ClickHouse/pull/12747) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed performance with large tuples, which are interpreted as functions in `IN` section. The case when user write `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed memory tracking for input_format_parallel_parsing (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). +* Fixed [#12293](https://github.com/ClickHouse/ClickHouse/issues/12293) allow push predicate when subquery contains with clause. [#12663](https://github.com/ClickHouse/ClickHouse/pull/12663) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572) fix bloom filter index with const expression. [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed SIGSEGV in StorageKafka when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). +* Fixed race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). +* Fixed bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). +* Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed performance issue, while reading from compact parts. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed the deadlock if `text_log` is enabled. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed possible segfault if StorageMerge. Closes [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). +* Fixed `TOTALS/ROLLUP/CUBE` for aggregate functions with `-State` and `Nullable` arguments. This fixes [#12163](https://github.com/ClickHouse/ClickHouse/issues/12163). [#12376](https://github.com/ClickHouse/ClickHouse/pull/12376) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed order of columns in `WITH FILL` modifier. Previously order of columns of `ORDER BY` statement wasn't respected. [#12306](https://github.com/ClickHouse/ClickHouse/pull/12306) ([Anton Popov](https://github.com/CurtizJ)). +* Avoid "bad cast" exception when there is an expression that filters data by virtual columns (like `_table` in `Merge` tables) or by "index" columns in system tables such as filtering by database name when querying from `system.tables`, and this expression returns `Nullable` type. This fixes [#12166](https://github.com/ClickHouse/ClickHouse/issues/12166). [#12305](https://github.com/ClickHouse/ClickHouse/pull/12305) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Show error after `TrieDictionary` failed to load. [#12290](https://github.com/ClickHouse/ClickHouse/pull/12290) ([Vitaly Baranov](https://github.com/vitlibar)). +* The function `arrayFill` worked incorrectly for empty arrays that may lead to crash. This fixes [#12263](https://github.com/ClickHouse/ClickHouse/issues/12263). [#12279](https://github.com/ClickHouse/ClickHouse/pull/12279) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Implement conversions to the common type for `LowCardinality` types. This allows to execute UNION ALL of tables with columns of LowCardinality and other columns. This fixes [#8212](https://github.com/ClickHouse/ClickHouse/issues/8212). This fixes [#4342](https://github.com/ClickHouse/ClickHouse/issues/4342). [#12275](https://github.com/ClickHouse/ClickHouse/pull/12275) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the behaviour when during multiple sequential inserts in `StorageFile` header for some special types was written more than once. This fixed [#6155](https://github.com/ClickHouse/ClickHouse/issues/6155). [#12197](https://github.com/ClickHouse/ClickHouse/pull/12197) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed logical functions for UInt8 values when they are not equal to 0 or 1. [#12196](https://github.com/ClickHouse/ClickHouse/pull/12196) ([Alexander Kazakov](https://github.com/Akazz)). +* Fixed `dictGet` arguments check during GROUP BY injective functions elimination. [#12179](https://github.com/ClickHouse/ClickHouse/pull/12179) ([Azat Khuzhin](https://github.com/azat)). +* Fixed wrong logic in `ALTER DELETE` that leads to deleting of records when condition evaluates to NULL. This fixes [#9088](https://github.com/ClickHouse/ClickHouse/issues/9088). This closes [#12106](https://github.com/ClickHouse/ClickHouse/issues/12106). [#12153](https://github.com/ClickHouse/ClickHouse/pull/12153) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed transform of query to send to external DBMS (e.g. MySQL, ODBC) in presense of aliases. This fixes [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential overflow in integer division. This fixes [#12119](https://github.com/ClickHouse/ClickHouse/issues/12119). [#12140](https://github.com/ClickHouse/ClickHouse/pull/12140) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential infinite loop in `greatCircleDistance`, `geoDistance`. This fixes [#12117](https://github.com/ClickHouse/ClickHouse/issues/12117). [#12137](https://github.com/ClickHouse/ClickHouse/pull/12137) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Avoid `There is no query` exception for materialized views with joins or with subqueries attached to system logs (system.query_log, metric_log, etc) or to engine=Buffer underlying table. [#12120](https://github.com/ClickHouse/ClickHouse/pull/12120) ([filimonov](https://github.com/filimonov)). +* Fixed performance for selects with `UNION` caused by wrong limit for the total number of threads. Fixes [#12030](https://github.com/ClickHouse/ClickHouse/issues/12030). [#12103](https://github.com/ClickHouse/ClickHouse/pull/12103) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed segfault with `-StateResample` combinators. [#12092](https://github.com/ClickHouse/ClickHouse/pull/12092) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed unnecessary limiting the number of threads for selects from `VIEW`. Fixes [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible crash while using wrong type for `PREWHERE`. Fixes [#12053](https://github.com/ClickHouse/ClickHouse/issues/12053), [#12060](https://github.com/ClickHouse/ClickHouse/issues/12060). [#12060](https://github.com/ClickHouse/ClickHouse/pull/12060) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed error `Expected single dictionary argument for function` for function `defaultValueOfArgumentType` with `LowCardinality` type. Fixes [#11808](https://github.com/ClickHouse/ClickHouse/issues/11808). [#12056](https://github.com/ClickHouse/ClickHouse/pull/12056) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed error `Cannot capture column` for higher-order functions with `Tuple(LowCardinality)` argument. Fixes [#9766](https://github.com/ClickHouse/ClickHouse/issues/9766). [#12055](https://github.com/ClickHouse/ClickHouse/pull/12055) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Parse tables metadata in parallel when loading database. This fixes slow server startup when there are large number of tables. [#12045](https://github.com/ClickHouse/ClickHouse/pull/12045) ([tavplubix](https://github.com/tavplubix)). +* Make `topK` aggregate function return Enum for Enum types. This fixes [#3740](https://github.com/ClickHouse/ClickHouse/issues/3740). [#12043](https://github.com/ClickHouse/ClickHouse/pull/12043) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed constraints check if constraint is a constant expression. This fixes [#11360](https://github.com/ClickHouse/ClickHouse/issues/11360). [#12042](https://github.com/ClickHouse/ClickHouse/pull/12042) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed incorrect comparison of tuples with `Nullable` columns. Fixes [#11985](https://github.com/ClickHouse/ClickHouse/issues/11985). [#12039](https://github.com/ClickHouse/ClickHouse/pull/12039) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed wrong result and potential crash when invoking function `if` with arguments of type `FixedString` with different sizes. This fixes [#11362](https://github.com/ClickHouse/ClickHouse/issues/11362). [#12021](https://github.com/ClickHouse/ClickHouse/pull/12021) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* A query with function `neighbor` as the only returned expression may return empty result if the function is called with offset `-9223372036854775808`. This fixes [#11367](https://github.com/ClickHouse/ClickHouse/issues/11367). [#12019](https://github.com/ClickHouse/ClickHouse/pull/12019) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential array size overflow in generateRandom that may lead to crash. This fixes [#11371](https://github.com/ClickHouse/ClickHouse/issues/11371). [#12013](https://github.com/ClickHouse/ClickHouse/pull/12013) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential floating point exception. This closes [#11378](https://github.com/ClickHouse/ClickHouse/issues/11378). [#12005](https://github.com/ClickHouse/ClickHouse/pull/12005) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed wrong setting name in log message at server startup. [#11997](https://github.com/ClickHouse/ClickHouse/pull/11997) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `Query parameter was not set` in `Values` format. Fixes [#11918](https://github.com/ClickHouse/ClickHouse/issues/11918). [#11936](https://github.com/ClickHouse/ClickHouse/pull/11936) ([tavplubix](https://github.com/tavplubix)). +* Keep aliases for substitutions in query (parametrized queries). This fixes [#11914](https://github.com/ClickHouse/ClickHouse/issues/11914). [#11916](https://github.com/ClickHouse/ClickHouse/pull/11916) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential floating point exception when parsing DateTime64. This fixes [#11374](https://github.com/ClickHouse/ClickHouse/issues/11374). [#11875](https://github.com/ClickHouse/ClickHouse/pull/11875) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed memory accounting via `HTTP` interface (can be significant with `wait_end_of_query=1`). [#11840](https://github.com/ClickHouse/ClickHouse/pull/11840) ([Azat Khuzhin](https://github.com/azat)). +* Fixed wrong result for `if()` with NULLs in condition. [#11807](https://github.com/ClickHouse/ClickHouse/pull/11807) ([Artem Zuikov](https://github.com/4ertus2)). +* Parse metadata stored in zookeeper before checking for equality. [#11739](https://github.com/ClickHouse/ClickHouse/pull/11739) ([Azat Khuzhin](https://github.com/azat)). +* Fixed `LIMIT n WITH TIES` usage together with `ORDER BY` statement, which contains aliases. [#11689](https://github.com/ClickHouse/ClickHouse/pull/11689) ([Anton Popov](https://github.com/CurtizJ)). +* Fix potential read of uninitialized memory in cache dictionary. [#10834](https://github.com/ClickHouse/ClickHouse/pull/10834) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Performance Improvement + +* Index not used for IN operator with literals", performance regression introduced around v19.3. This fixes "[#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). + + ### ClickHouse release v20.3.12.112-lts 2020-06-25 #### Bug Fix From 6267daa75095e44f358f6ecb8c6430a37e25b081 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 12 Aug 2020 00:03:01 +0300 Subject: [PATCH 263/374] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 927bba1198f..4e1c1d59b99 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -8,7 +8,7 @@ * Added storage `RabbitMQ`. [#11069](https://github.com/ClickHouse/ClickHouse/pull/11069) ([Kseniia Sumarokova](https://github.com/kssenii)). * Implemented PostgreSQL-like `ILIKE` operator for [#11710](https://github.com/ClickHouse/ClickHouse/issues/11710). [#12125](https://github.com/ClickHouse/ClickHouse/pull/12125) ([Mike](https://github.com/myrrc)). * Supported RIGHT and FULL JOIN with `SET join_algorithm = 'partial_merge'`. Only ALL strictness is allowed (ANY, SEMI, ANTI, ASOF are not). [#12118](https://github.com/ClickHouse/ClickHouse/pull/12118) ([Artem Zuikov](https://github.com/4ertus2)). -* Added a function `initializedAggregation` to initialize an aggregation based on a single value. [#12109](https://github.com/ClickHouse/ClickHouse/pull/12109) ([Guillaume Tassery](https://github.com/YiuRULE)). +* Added a function `initializeAggregation` to initialize an aggregation based on a single value. [#12109](https://github.com/ClickHouse/ClickHouse/pull/12109) ([Guillaume Tassery](https://github.com/YiuRULE)). * Supported `ALTER TABLE ... [ADD|MODIFY] COLUMN ... FIRST` [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#12073](https://github.com/ClickHouse/ClickHouse/pull/12073) ([Winter Zhang](https://github.com/zhang2014)). * Added function `parseDateTimeBestEffortUS`. [#12028](https://github.com/ClickHouse/ClickHouse/pull/12028) ([flynn](https://github.com/ucasFL)). * Support format `ORC` for output (was supported only for input). [#11662](https://github.com/ClickHouse/ClickHouse/pull/11662) ([Kruglov Pavel](https://github.com/Avogar)). From e4da1d8e51a258130599c79bd1279a6dd967bb74 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 11 Aug 2020 23:58:39 +0300 Subject: [PATCH 264/374] fix deadlock and logical error in replication queue --- .../MergeTree/ReplicatedMergeTreeLogEntry.h | 1 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 74 +++++++++++++++---- .../MergeTree/ReplicatedMergeTreeQueue.h | 3 + ...ent_move_partition_from_table_zookeeper.sh | 10 +++ 4 files changed, 72 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 641f10722fe..d2b42edb7bc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -161,6 +161,7 @@ struct ReplicatedMergeTreeLogEntryData /// Access under queue_mutex, see ReplicatedMergeTreeQueue. bool currently_executing = false; /// Whether the action is executing now. + bool removed_by_other_entry = false; /// These several fields are informational only (for viewing by the user using system tables). /// Access under queue_mutex, see ReplicatedMergeTreeQueue. size_t num_tries = 0; /// The number of attempts to perform the action (since the server started, including the running one). diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 1b19f792679..8e2c3752212 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -341,38 +341,50 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep std::optional max_processed_insert_time_changed; bool found = false; + bool need_remove_from_zk = true; size_t queue_size = 0; /// First remove from memory then from ZooKeeper { std::unique_lock lock(state_mutex); - - /// Remove the job from the queue in the RAM. - /// You can not just refer to a pre-saved iterator, because someone else might be able to delete the task. - /// Why do we view the queue from the end? - /// - because the task for execution first is moved to the end of the queue, so that in case of failure it remains at the end. - for (Queue::iterator it = queue.end(); it != queue.begin();) + if (entry->removed_by_other_entry) { - --it; - if (*it == entry) + need_remove_from_zk = false; + queue_size = queue.size(); + } + else + { + /// Remove the job from the queue in the RAM. + /// You can not just refer to a pre-saved iterator, because someone else might be able to delete the task. + /// Why do we view the queue from the end? + /// - because the task for execution first is moved to the end of the queue, so that in case of failure it remains at the end. + for (Queue::iterator it = queue.end(); it != queue.begin();) { - found = true; - updateStateOnQueueEntryRemoval( - entry, /* is_successful = */ true, - min_unprocessed_insert_time_changed, max_processed_insert_time_changed, lock); + --it; - queue.erase(it); - queue_size = queue.size(); - break; + if (*it == entry) + { + found = true; + updateStateOnQueueEntryRemoval( + entry, /* is_successful = */ true, + min_unprocessed_insert_time_changed, max_processed_insert_time_changed, lock); + + queue.erase(it); + queue_size = queue.size(); + break; + } } } } - if (!found) + if (!found && need_remove_from_zk) throw Exception("Can't find " + entry->znode_name + " in the memory queue. It is a bug", ErrorCodes::LOGICAL_ERROR); notifySubscribers(queue_size); + if (!need_remove_from_zk) + return; + auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name); if (code != Coordination::Error::ZOK) LOG_ERROR(log, "Couldn't remove {}/queue/{}: {}. This shouldn't happen often.", replica_path, entry->znode_name, Coordination::errorMessage(code)); @@ -843,6 +855,7 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( const MergeTreePartInfo & part_info, const ReplicatedMergeTreeLogEntryData & current) { + /// TODO is it possible to simplify it? Queue to_wait; size_t removed_entries = 0; std::optional min_unprocessed_insert_time_changed; @@ -850,6 +863,10 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( /// Remove operations with parts, contained in the range to be deleted, from the queue. std::unique_lock lock(state_mutex); + + [[maybe_unused]] bool called_from_alter_query_directly = current.replace_range_entry && current.replace_range_entry->columns_version < 0; + assert(currently_executing_drop_or_replace_range || called_from_alter_query_directly); + for (Queue::iterator it = queue.begin(); it != queue.end();) { auto type = (*it)->type; @@ -861,12 +878,17 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( if ((*it)->currently_executing) to_wait.push_back(*it); auto code = zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name); + /// FIXME it's probably unsafe to remove entries non-atomically + /// when this method called directly from alter query (not from replication queue task), + /// because entries will be lost if ALTER fails. if (code != Coordination::Error::ZOK) LOG_INFO(log, "Couldn't remove {}: {}", replica_path + "/queue/" + (*it)->znode_name, Coordination::errorMessage(code)); updateStateOnQueueEntryRemoval( *it, /* is_successful = */ false, min_unprocessed_insert_time_changed, max_processed_insert_time_changed, lock); + + (*it)->removed_by_other_entry = true; queue.erase(it++); ++removed_entries; } @@ -1099,6 +1121,16 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( } } + if (entry.type == LogEntry::DROP_RANGE || entry.type == LogEntry::REPLACE_RANGE) + { + /// DROP_RANGE and REPLACE_RANGE entries remove other entries, which produce parts in the range. + /// If such part producing operations are currently executing, then DROP/REPLACE RANGE wait them to finish. + /// Deadlock is possible if multiple DROP/REPLACE RANGE entries are executing in parallel and wait each other. + /// See also removePartProducingOpsInRange(...) and ReplicatedMergeTreeQueue::CurrentlyExecuting. + if (currently_executing_drop_or_replace_range) + return false; + } + return true; } @@ -1129,6 +1161,11 @@ Int64 ReplicatedMergeTreeQueue::getCurrentMutationVersion(const String & partiti ReplicatedMergeTreeQueue::CurrentlyExecuting::CurrentlyExecuting(const ReplicatedMergeTreeQueue::LogEntryPtr & entry_, ReplicatedMergeTreeQueue & queue_) : entry(entry_), queue(queue_) { + if (entry->type == ReplicatedMergeTreeLogEntry::DROP_RANGE || entry->type == ReplicatedMergeTreeLogEntry::REPLACE_RANGE) + { + assert(!queue.currently_executing_drop_or_replace_range); + queue.currently_executing_drop_or_replace_range = true; + } entry->currently_executing = true; ++entry->num_tries; entry->last_attempt_time = time(nullptr); @@ -1162,6 +1199,11 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting() { std::lock_guard lock(queue.state_mutex); + if (entry->type == ReplicatedMergeTreeLogEntry::DROP_RANGE || entry->type == ReplicatedMergeTreeLogEntry::REPLACE_RANGE) + { + assert(queue.currently_executing_drop_or_replace_range); + queue.currently_executing_drop_or_replace_range = false; + } entry->currently_executing = false; entry->execution_complete.notify_all(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 030ae9cfc05..76f84da1ae8 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -87,6 +87,9 @@ private: /// Index of the first log entry that we didn't see yet. Int64 log_pointer = 0; + /// Avoid parallel execution of queue enties, which may remove other entries from the queue. + bool currently_executing_drop_or_replace_range = false; + /** What will be the set of active parts after executing all log entries up to log_pointer. * Used to determine which merges can be assigned (see ReplicatedMergeTreeMergePredicate) */ diff --git a/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh index 7870c582fb9..313f825a400 100755 --- a/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh @@ -43,11 +43,20 @@ function thread4() done } +function thread5() +{ + while true; + do + $CLICKHOUSE_CLIENT --query="ALTER TABLE src MOVE PARTITION 1 TO TABLE dst;" --query_id=query5 + done +} + # https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout export -f thread1; export -f thread2; export -f thread3; export -f thread4; +export -f thread5; TIMEOUT=30 @@ -55,6 +64,7 @@ timeout $TIMEOUT bash -c thread1 2> /dev/null & timeout $TIMEOUT bash -c thread2 2> /dev/null & timeout $TIMEOUT bash -c thread3 2> /dev/null & timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & wait From 829f7e65de397071cd6ee49a428b030f55dad7e9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 12 Aug 2020 00:59:46 +0300 Subject: [PATCH 265/374] try fix sync, add experimetal flag --- src/Core/Settings.h | 1 + src/DataStreams/ya.make | 1 - src/Interpreters/InterpreterCreateQuery.cpp | 6 ++++++ .../configs/remote_servers.xml | 12 ------------ .../configs/users.xml | 18 ++++++++++++++++++ .../test_materialize_mysql_database/test.py | 2 +- 6 files changed, 26 insertions(+), 14 deletions(-) delete mode 100644 tests/integration/test_materialize_mysql_database/configs/remote_servers.xml create mode 100644 tests/integration/test_materialize_mysql_database/configs/users.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ff137a54381..06eedf4f337 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -379,6 +379,7 @@ class IColumn; M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ + M(Bool, allow_experimental_database_materialize_mysql, false, "Allow to create database with Engine=MaterializeMySQL(...).", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ diff --git a/src/DataStreams/ya.make b/src/DataStreams/ya.make index dcada250d38..a9bbd987ff0 100644 --- a/src/DataStreams/ya.make +++ b/src/DataStreams/ya.make @@ -11,7 +11,6 @@ NO_COMPILER_WARNINGS() SRCS( AddingDefaultBlockOutputStream.cpp AddingDefaultsBlockInputStream.cpp - AddingVersionsBlockOutputStream.cpp AsynchronousBlockInputStream.cpp BlockIO.cpp BlockStreamProfileInfo.cpp diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 7e5857c8440..2e3b5daad6d 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -168,6 +168,12 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) metadata_path = metadata_path / "metadata" / database_name_escaped; } + if (create.storage->engine->name == "MaterializeMySQL" && !context.getSettingsRef().allow_experimental_database_materialize_mysql && !internal) + { + throw Exception("MaterializeMySQL is an experimental database engine. " + "Enable allow_experimental_database_materialize_mysql to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); + } + DatabasePtr database = DatabaseFactory::get(create, metadata_path / "", context); if (create.uuid != UUIDHelpers::Nil) diff --git a/tests/integration/test_materialize_mysql_database/configs/remote_servers.xml b/tests/integration/test_materialize_mysql_database/configs/remote_servers.xml deleted file mode 100644 index de8e5865f12..00000000000 --- a/tests/integration/test_materialize_mysql_database/configs/remote_servers.xml +++ /dev/null @@ -1,12 +0,0 @@ - - - - - - node1 - 9000 - - - - - diff --git a/tests/integration/test_materialize_mysql_database/configs/users.xml b/tests/integration/test_materialize_mysql_database/configs/users.xml new file mode 100644 index 00000000000..7e5c9cce7ba --- /dev/null +++ b/tests/integration/test_materialize_mysql_database/configs/users.xml @@ -0,0 +1,18 @@ + + + + + 1 + + + + + + + + ::/0 + + default + + + diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 26f8b43da0e..08baf87e69f 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -11,7 +11,7 @@ from helpers.cluster import ClickHouseCluster SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) cluster = ClickHouseCluster(__file__) -clickhouse_node = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], with_mysql=False) +clickhouse_node = cluster.add_instance('node1', config_dir="configs", with_mysql=False) @pytest.fixture(scope="module") From c6428f851a7b2ab04bfa16f1cd1dc7390f9589f1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 12 Aug 2020 01:04:38 +0300 Subject: [PATCH 266/374] Minor fixes in clickhouse install --- programs/install/Install.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 4a76608b9d7..9b35e6e83be 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -175,7 +175,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) catch (const Exception & e) { if (e.code() == ErrorCodes::CANNOT_OPEN_FILE && geteuid() != 0) - std::cerr << "Install must be run as root: sudo ./clickhouse install"; + std::cerr << "Install must be run as root: sudo ./clickhouse install\n"; throw; } @@ -583,7 +583,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } catch (...) { - std::cerr << getCurrentExceptionMessage(false); + std::cerr << getCurrentExceptionMessage(false) << '\n'; return getCurrentExceptionCode(); } @@ -782,7 +782,7 @@ int mainEntryClickHouseStart(int argc, char ** argv) } catch (...) { - std::cerr << getCurrentExceptionMessage(false); + std::cerr << getCurrentExceptionMessage(false) << '\n'; return getCurrentExceptionCode(); } } @@ -816,7 +816,7 @@ int mainEntryClickHouseStop(int argc, char ** argv) } catch (...) { - std::cerr << getCurrentExceptionMessage(false); + std::cerr << getCurrentExceptionMessage(false) << '\n'; return getCurrentExceptionCode(); } } @@ -850,7 +850,7 @@ int mainEntryClickHouseStatus(int argc, char ** argv) } catch (...) { - std::cerr << getCurrentExceptionMessage(false); + std::cerr << getCurrentExceptionMessage(false) << '\n'; return getCurrentExceptionCode(); } } @@ -893,7 +893,7 @@ int mainEntryClickHouseRestart(int argc, char ** argv) } catch (...) { - std::cerr << getCurrentExceptionMessage(false); + std::cerr << getCurrentExceptionMessage(false) << '\n'; return getCurrentExceptionCode(); } } From cb031d3548a8304f7eced7e20877c577c9cbcae6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 12 Aug 2020 01:48:25 +0300 Subject: [PATCH 267/374] Minor changes --- programs/install/Install.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 9b35e6e83be..f0d51ef95c9 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -469,6 +469,9 @@ int mainEntryClickHouseInstall(int argc, char ** argv) executeScript(command); } + /// All users are allowed to read pid file (for clickhouse status command). + fs::permissions(pid_path, fs::perms::owner_all | fs::perms::group_read | fs::perms::others_read, fs::perm_options::replace); + /// Other users in clickhouse group are allowed to read and even delete logs. fs::permissions(log_path, fs::perms::owner_all | fs::perms::group_all, fs::perm_options::replace); From e65a59c400f5a3cecc4f5012612e37730953613b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 12 Aug 2020 01:49:38 +0300 Subject: [PATCH 268/374] Added comments --- programs/install/Install.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index f0d51ef95c9..8e6365ac5d5 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -144,6 +144,9 @@ int mainEntryClickHouseInstall(int argc, char ** argv) /// Copy binary to the destination directory. + /// TODO An option to link instead of copy - useful for developers. + /// TODO Check if the binary is the same. + size_t binary_size = fs::file_size(binary_self_path); fs::path prefix = fs::path(options["prefix"].as()); From 2bcc850533c61c970fc7e71f79c155900eadfa4f Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Wed, 12 Aug 2020 02:00:21 +0300 Subject: [PATCH 269/374] Update CMakeLists.txt --- contrib/aws-s3-cmake/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/aws-s3-cmake/CMakeLists.txt b/contrib/aws-s3-cmake/CMakeLists.txt index d889fca197f..b558c4732aa 100644 --- a/contrib/aws-s3-cmake/CMakeLists.txt +++ b/contrib/aws-s3-cmake/CMakeLists.txt @@ -91,7 +91,6 @@ set_property(TARGET aws_s3_checksums PROPERTY C_STANDARD 99) add_library(aws_s3 ${S3_UNIFIED_SRC}) -target_compile_definitions(aws_s3 PUBLIC -DENABLE_CURL_CLIENT) target_compile_definitions(aws_s3 PUBLIC "AWS_SDK_VERSION_MAJOR=1") target_compile_definitions(aws_s3 PUBLIC "AWS_SDK_VERSION_MINOR=7") target_compile_definitions(aws_s3 PUBLIC "AWS_SDK_VERSION_PATCH=231") From fb527e8677c69673c2d006f2e52de240318a9df7 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Wed, 12 Aug 2020 02:03:53 +0300 Subject: [PATCH 270/374] Update CMakeLists.txt --- contrib/aws-s3-cmake/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/aws-s3-cmake/CMakeLists.txt b/contrib/aws-s3-cmake/CMakeLists.txt index b558c4732aa..b85d91f88c7 100644 --- a/contrib/aws-s3-cmake/CMakeLists.txt +++ b/contrib/aws-s3-cmake/CMakeLists.txt @@ -18,7 +18,6 @@ file(GLOB AWS_CORE_SOURCES "${AWS_CORE_LIBRARY_DIR}/source/client/*.cpp" "${AWS_CORE_LIBRARY_DIR}/source/http/*.cpp" "${AWS_CORE_LIBRARY_DIR}/source/http/standard/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/http/curl/*.cpp" "${AWS_CORE_LIBRARY_DIR}/source/config/*.cpp" "${AWS_CORE_LIBRARY_DIR}/source/external/cjson/*.cpp" "${AWS_CORE_LIBRARY_DIR}/source/external/tinyxml2/*.cpp" From 11c85065ef15e2334146b73f668490c2017bd95f Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Wed, 12 Aug 2020 02:05:20 +0300 Subject: [PATCH 271/374] Update CMakeLists.txt --- contrib/aws-s3-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/aws-s3-cmake/CMakeLists.txt b/contrib/aws-s3-cmake/CMakeLists.txt index b85d91f88c7..02dee91c70c 100644 --- a/contrib/aws-s3-cmake/CMakeLists.txt +++ b/contrib/aws-s3-cmake/CMakeLists.txt @@ -100,4 +100,4 @@ if (OPENSSL_FOUND) target_link_libraries(aws_s3 PRIVATE ${OPENSSL_LIBRARIES}) endif() -target_link_libraries(aws_s3 PRIVATE aws_s3_checksums curl) +target_link_libraries(aws_s3 PRIVATE aws_s3_checksums) From 073fd3099260a78103299caf7f5c9374da15aee1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 12 Aug 2020 02:19:17 +0300 Subject: [PATCH 272/374] More gentle font size in docs --- website/css/bootstrap.css | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/website/css/bootstrap.css b/website/css/bootstrap.css index f9fad939795..b52c19c2b8a 100644 --- a/website/css/bootstrap.css +++ b/website/css/bootstrap.css @@ -363,25 +363,25 @@ h1, h2, h3, h4, h5, h6, } h1, .h1 { - font-size: 2.5rem; -} - -h2, .h2 { font-size: 2rem; } -h3, .h3 { - font-size: 1.75rem; -} - -h4, .h4 { +h2, .h2 { font-size: 1.5rem; } -h5, .h5 { +h3, .h3 { font-size: 1.25rem; } +h4, .h4 { + font-size: 1rem; +} + +h5, .h5 { + font-size: 1rem; +} + h6, .h6 { font-size: 1rem; } From 2ee6f1484c0a41e60fd1e7d8dea48dfd477be948 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 12 Aug 2020 02:22:00 +0300 Subject: [PATCH 273/374] Fix bad documentation (progress: 1%) --- docs/en/sql-reference/functions/string-replace-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 731215277f5..1f399e02052 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -88,7 +88,7 @@ SELECT replaceRegexpAll('Hello, World!', '^', 'here: ') AS res ## regexpQuoteMeta(s) {#regexpquotemetas} The function adds a backslash before some predefined characters in the string. -Predefined characters: ‘0’, ‘\\’, ‘\|’, ‘(’, ‘)’, ‘^’, ‘$’, ‘.’, ‘\[’, ’\]’, ‘?’, ’\*‘,’+‘,’{‘,’:‘,’-’. +Predefined characters: `\0`, `\\`, `|`, `(`, `)`, `^`, `$`, `.`, `[`, `]`, `?`, `*`, `+`, `{`, `:`, `-`. This implementation slightly differs from re2::RE2::QuoteMeta. It escapes zero byte as \\0 instead of 00 and it escapes only required characters. For more information, see the link: [RE2](https://github.com/google/re2/blob/master/re2/re2.cc#L473) From d341ec8c3d1e7aefe0fc939130378fd755d73778 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 12 Aug 2020 02:23:11 +0300 Subject: [PATCH 274/374] Fix misleading, bad and wrong documentation. --- docs/en/sql-reference/functions/string-replace-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 1f399e02052..8905500995c 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -89,7 +89,7 @@ SELECT replaceRegexpAll('Hello, World!', '^', 'here: ') AS res The function adds a backslash before some predefined characters in the string. Predefined characters: `\0`, `\\`, `|`, `(`, `)`, `^`, `$`, `.`, `[`, `]`, `?`, `*`, `+`, `{`, `:`, `-`. -This implementation slightly differs from re2::RE2::QuoteMeta. It escapes zero byte as \\0 instead of 00 and it escapes only required characters. +This implementation slightly differs from re2::RE2::QuoteMeta. It escapes zero byte as `\0` instead of `\x00` and it escapes only required characters. For more information, see the link: [RE2](https://github.com/google/re2/blob/master/re2/re2.cc#L473) [Original article](https://clickhouse.tech/docs/en/query_language/functions/string_replace_functions/) From df9d5d4e29e8af07fa050dd06bdabd8a611b66f2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 12 Aug 2020 02:28:45 +0300 Subject: [PATCH 275/374] Better documentation --- website/templates/docs/content.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/templates/docs/content.html b/website/templates/docs/content.html index 29db92e377a..6a9bd1f69d9 100644 --- a/website/templates/docs/content.html +++ b/website/templates/docs/content.html @@ -1,4 +1,4 @@ -
+
{% if not single_page %} {% set ancestors = page.ancestors|reverse|list %} From ffcbfc383bc49afd276573ed7b7d6ccbac8373a4 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 12 Aug 2020 09:30:40 +0800 Subject: [PATCH 276/374] remove obsolete comments --- src/Functions/FunctionMathBinaryFloat64.h | 7 ------- src/Functions/FunctionMathUnary.h | 7 ------- 2 files changed, 14 deletions(-) diff --git a/src/Functions/FunctionMathBinaryFloat64.h b/src/Functions/FunctionMathBinaryFloat64.h index 023c23cde72..b11716269ba 100644 --- a/src/Functions/FunctionMathBinaryFloat64.h +++ b/src/Functions/FunctionMathBinaryFloat64.h @@ -13,13 +13,6 @@ # include "config_functions.h" #endif -/** More efficient implementations of mathematical functions are possible when using a separate library. - * Disabled due to license compatibility limitations. - * To enable: download http://www.agner.org/optimize/vectorclass.zip and unpack to contrib/vectorclass - * Then rebuild with -DENABLE_VECTORCLASS=1 - */ - - namespace DB { diff --git a/src/Functions/FunctionMathUnary.h b/src/Functions/FunctionMathUnary.h index ae1192c61df..ab58b3739ae 100644 --- a/src/Functions/FunctionMathUnary.h +++ b/src/Functions/FunctionMathUnary.h @@ -12,13 +12,6 @@ # include "config_functions.h" #endif -/** More efficient implementations of mathematical functions are possible when using a separate library. - * Disabled due to license compatibility limitations. - * To enable: download http://www.agner.org/optimize/vectorclass.zip and unpack to contrib/vectorclass - * Then rebuild with -DENABLE_VECTORCLASS=1 - */ - - /** FastOps is a fast vector math library from Mikhail Parakhin (former Yandex CTO), * Enabled by default. */ From cf4ad2f4babf718907980f1279c468d4d3c7c25c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 12 Aug 2020 05:56:56 +0300 Subject: [PATCH 277/374] Preparing new benchmark for MonetDB --- benchmark/monetdb/benchmark.sh | 9 + benchmark/monetdb/conf.sh | 4 - benchmark/monetdb/define_schema.sql | 3 - benchmark/monetdb/log/log_100m | 24304 ----------------- benchmark/monetdb/log/log_100m_1 | 3273 --- benchmark/monetdb/log/log_100m_corrected | 1295 - benchmark/monetdb/log/log_100m_corrected_1 | Bin 38392 -> 0 bytes benchmark/monetdb/log/log_100m_corrected_2 | 3643 --- benchmark/monetdb/log/log_10m | 2250 -- benchmark/monetdb/log/log_10m_corrected | 5498 ---- benchmark/monetdb/log/log_10m_corrected_1 | 3199 --- benchmark/monetdb/log/log_upload_100m | 62 - benchmark/monetdb/log/log_upload_1b | 62 - benchmark/monetdb/queries.sql | 154 +- benchmark/monetdb/{expect.tcl => send-query} | 9 +- 15 files changed, 56 insertions(+), 43709 deletions(-) create mode 100755 benchmark/monetdb/benchmark.sh delete mode 100644 benchmark/monetdb/conf.sh delete mode 100644 benchmark/monetdb/define_schema.sql delete mode 100644 benchmark/monetdb/log/log_100m delete mode 100644 benchmark/monetdb/log/log_100m_1 delete mode 100644 benchmark/monetdb/log/log_100m_corrected delete mode 100644 benchmark/monetdb/log/log_100m_corrected_1 delete mode 100644 benchmark/monetdb/log/log_100m_corrected_2 delete mode 100644 benchmark/monetdb/log/log_10m delete mode 100644 benchmark/monetdb/log/log_10m_corrected delete mode 100644 benchmark/monetdb/log/log_10m_corrected_1 delete mode 100644 benchmark/monetdb/log/log_upload_100m delete mode 100644 benchmark/monetdb/log/log_upload_1b rename benchmark/monetdb/{expect.tcl => send-query} (62%) mode change 100644 => 100755 diff --git a/benchmark/monetdb/benchmark.sh b/benchmark/monetdb/benchmark.sh new file mode 100755 index 00000000000..ad98bcbe8f3 --- /dev/null +++ b/benchmark/monetdb/benchmark.sh @@ -0,0 +1,9 @@ +#!/bin/bash + +grep -v -P '^#' queries.sql | sed -e 's/{table}/hits/' | while read query; do + + echo "$query"; + for i in {1..3}; do + ./send-query "$query" 2>&1 | grep -P '\d+ tuple|clk: |unknown|overflow|error'; + done; +done; diff --git a/benchmark/monetdb/conf.sh b/benchmark/monetdb/conf.sh deleted file mode 100644 index 8b9d2191733..00000000000 --- a/benchmark/monetdb/conf.sh +++ /dev/null @@ -1,4 +0,0 @@ -CONF_DIR=/home/kartavyy/benchmark/monetdb -expect_file=$CONF_DIR/expect.tcl -test_file=$CONF_DIR/queries.sql -etc_init_d_service=/etc/init.d/monetdb5-sql diff --git a/benchmark/monetdb/define_schema.sql b/benchmark/monetdb/define_schema.sql deleted file mode 100644 index 72de2ab028b..00000000000 --- a/benchmark/monetdb/define_schema.sql +++ /dev/null @@ -1,3 +0,0 @@ -create table hits_10m ( WatchID BIGINT, JavaEnable SMALLINT, Title VARCHAR(1400), GoodEvent SMALLINT, EventTime TIMESTAMP, EventDate DATE, CounterID BIGINT, ClientIP BIGINT, RegionID BIGINT, UserID BIGINT, CounterClass TINYINT, OS SMALLINT, UserAgent SMALLINT, URL VARCHAR(7800), Referer VARCHAR(3125), Refresh TINYINT, RefererCategoryID INT, RefererRegionID BIGINT, URLCategoryID INT, URLRegionID BIGINT, ResolutionWidth INT, ResolutionHeight INT, ResolutionDepth SMALLINT, FlashMajor SMALLINT, FlashMinor SMALLINT, FlashMinor2 VARCHAR(256), NetMajor SMALLINT, NetMinor SMALLINT, UserAgentMajor INT, UserAgentMinor CHAR(2), CookieEnable SMALLINT, JavascriptEnable SMALLINT, IsMobile SMALLINT, MobilePhone SMALLINT, MobilePhoneModel VARCHAR(80), Params VARCHAR(2925), IPNetworkID BIGINT, TraficSourceID SMALLINT, SearchEngineID INT, SearchPhrase VARCHAR(2008), AdvEngineID SMALLINT, IsArtifical SMALLINT, WindowClientWidth INT, WindowClientHeight INT, ClientTimeZone INTEGER, ClientEventTime TIMESTAMP, SilverlightVersion1 SMALLINT, SilverlightVersion2 SMALLINT, SilverlightVersion3 BIGINT, SilverlightVersion4 INT, PageCharset VARCHAR(80), CodeVersion BIGINT, IsLink SMALLINT, IsDownload SMALLINT, IsNotBounce SMALLINT, FUniqID BIGINT, OriginalURL VARCHAR(8181), HID BIGINT, IsOldCounter SMALLINT, IsEvent SMALLINT, IsParameter SMALLINT, DontCountHits SMALLINT, WithHash SMALLINT, HitColor CHAR(1), LocalEventTime TIMESTAMP, Age SMALLINT, Sex SMALLINT, Income SMALLINT, Interests INT, Robotness SMALLINT, RemoteIP BIGINT, WindowName INT, OpenerName INT, HistoryLength SMALLINT, BrowserLanguage CHAR(2), BrowserCountry CHAR(2), SocialNetwork VARCHAR(128), SocialAction VARCHAR(128), HTTPError INT, SendTiming BIGINT, DNSTiming BIGINT, ConnectTiming BIGINT, ResponseStartTiming BIGINT, ResponseEndTiming BIGINT, FetchTiming BIGINT, SocialSourceNetworkID SMALLINT, SocialSourcePage VARCHAR(256), ParamPrice BIGINT, ParamOrderID VARCHAR(80), ParamCurrency CHAR(3), ParamCurrencyID INT, OpenstatServiceName VARCHAR(80), OpenstatCampaignID VARCHAR(512), OpenstatAdID VARCHAR(80), OpenstatSourceID VARCHAR(256), UTMSource VARCHAR(256), UTMMedium VARCHAR(256), UTMCampaign VARCHAR(407), UTMContent VARCHAR(256), UTMTerm VARCHAR(437), FromTag VARCHAR(428), HasGCLID SMALLINT, RefererHash BIGINT, URLHash BIGINT, CLID BIGINT, UserIDHash BIGINT ); CREATE INDEX hits_10m_ind ON hits_10m(CounterID, EventDate, UserIDHash, EventTime); - -COPY INTO hits_10m FROM ('/opt/dump/dump_0.3/dump_hits_10m_meshed_utf8.tsv') DELIMITERS '\t'; \ No newline at end of file diff --git a/benchmark/monetdb/log/log_100m b/benchmark/monetdb/log/log_100m deleted file mode 100644 index de5758ca394..00000000000 --- a/benchmark/monetdb/log/log_100m +++ /dev/null @@ -1,24304 +0,0 @@ -start time: Thu Sep 12 21:41:01 MSK 2013 -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT count(*) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m; -+-----------+ -+===========+ -+-----------+ -1 tuple (34.751ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT count(*) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m; -+-----------+ -+===========+ -+-----------+ -1 tuple (40.370ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT count(*) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m; -+-----------+ -+===========+ -+-----------+ -1 tuple (3.464ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT count(*) FROM hits_100m WHERE AdvEngineID not like 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m WHERE AdvEngineID not like 0; -+--------+ -+========+ -+--------+ -1 tuple (54.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT count(*) FROM hits_100m WHERE AdvEngineID not like 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m WHERE AdvEngineID not like 0; -+--------+ -+========+ -+--------+ -1 tuple (52.8s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT count(*) FROM hits_100m WHERE AdvEngineID not like 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m WHERE AdvEngineID not like 0; -+--------+ -+========+ -+--------+ -1 tuple (53.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -+---------+-----------+--------------------------+ -+=========+===========+==========================+ -+---------+-----------+--------------------------+ -1 tuple (1.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -+---------+-----------+--------------------------+ -+=========+===========+==========================+ -+---------+-----------+--------------------------+ -1 tuple (152.773ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -+---------+-----------+--------------------------+ -+=========+===========+==========================+ -+---------+-----------+--------------------------+ -1 tuple (148.570ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT sum(UserID) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_100m; -overflow in calculation. -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT sum(UserID) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_100m; -overflow in calculation. -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT sum(UserID) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_100m; -overflow in calculation. -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT count(DISTINCT UserID) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_100m; -+----------+ -+==========+ -+----------+ -1 tuple (7.8s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT count(DISTINCT UserID) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_100m; -+----------+ -+==========+ -+----------+ -1 tuple (5.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT count(DISTINCT UserID) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_100m; -+----------+ -+==========+ -+----------+ -1 tuple (5.7s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -+---------+ -+=========+ -+---------+ -1 tuple (15.6s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -+---------+ -+=========+ -+---------+ -1 tuple (11.9s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -+---------+ -+=========+ -+---------+ -1 tuple (12.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT min(EventDate), max(EventDate) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_100m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (1.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT min(EventDate), max(EventDate) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_100m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (35.115ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT min(EventDate), max(EventDate) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_100m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (36.394ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID not like 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID not like 0 \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID not like 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID not like 0 \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID not like 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID not like 0 \q --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY Regi \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY Regi \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY Regi \q --- агрегация, среднее количество ключей.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), c \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), c \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), c \q --- агрегация, среднее количество ключей, несколько агрегатных функций.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel not like '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE <, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel not like \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel not like '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE <, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel not like \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel not like '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE <, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel not like \q --- мощная фильтрация по строкам, затем агрегация по строкам.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel not like '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hi -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel not like '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hi -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel not like '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hi -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase not like \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase not like \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase not like \q --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE Sea \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE Sea \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE Sea \q --- агрегация чуть сложнее.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE Search \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE Search \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE Search \q --- агрегация по числу и строке, большое количество ключей.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) < FROM hits_100m GROUP BY UserID ORDER BY count(*) D ESC LIMIT 10; -+---------------------+-------+ -+=====================+=======+ -+---------------------+-------+ -10 tuples (7.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) < FROM hits_100m GROUP BY UserID ORDER BY count(*) D ESC LIMIT 10; -+---------------------+-------+ -+=====================+=======+ -+---------------------+-------+ -10 tuples (8.4s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) < FROM hits_100m GROUP BY UserID ORDER BY count(*) D ESC LIMIT 10; -+---------------------+-------+ -+=====================+=======+ -+---------------------+-------+ -10 tuples (7.2s) -sql>\q --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, Sea \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, Sea \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, Sea -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, Sea \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, Sea \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, Sea \q --- то же самое, но без сортировки.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count( <(minute from EventTime) AS m, SearchPhrase, count(* ) FROM hits_100m GROUP BY < m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase O < FROM hits_100m GROUP BY UserID, m, SearchPhrase OR DER BY count(*) DESC LIMI -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count( <(minute from EventTime) AS m, SearchPhrase, count(* ) FROM hits_100m GROUP BY < m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase O < FROM hits_100m GROUP BY UserID, m, SearchPhrase OR DER BY count(*) DESC LIMI set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count( <(minute from EventTime) AS m, SearchPhrase, count(* ) FROM hits_100m GROUP BY < m, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, m, SearchPhrase O < FROM hits_100m GROUP BY UserID, m, SearchPhrase OR DER BY count(*) DESC LIMI \q --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT UserID FROM hits_100m WHERE UserID = 1234567890; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID FROM hits_100m WHERE UserID = 1234567890; -+--------+ -+========+ -+--------+ -0 tuples (3.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT UserID FROM hits_100m WHERE UserID = 1234567890; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID FROM hits_100m WHERE UserID = 1234567890; -+--------+ -+========+ -+--------+ -0 tuples (0.808ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT UserID FROM hits_100m WHERE UserID = 1234567890; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID FROM hits_100m WHERE UserID = 1234567890; -+--------+ -+========+ -+--------+ -0 tuples (0.629ms) -sql>\q --- мощная фильтрация по столбцу типа UInt64.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -+------+ -+======+ -+------+ -1 tuple (25.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -+------+ -+======+ -+------+ -1 tuple (2.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m WHERE URL LIKE '%metrika%'; -+------+ -+======+ -+------+ -1 tuple (2.5s) -sql>\q --- фильтрация по поиску подстроки в строке.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%m \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%m \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%metrika%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), count(*) FROM hits_100m WHERE URL LIKE '%m \q --- вынимаем большие столбцы, фильтрация по строке.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_100m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT \q --- чуть больше столбцы.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIM < WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMI T 10; -+---------------------+------------+---------------------------------------------------+-----------+----------------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+----------------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+----------------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+=====================+============+===================================================+===========+============================+============+===========+============+==========+=====================+==============+======+===========+================================================================+=========+=========+===================+=================+===============+=============+=================+==================+=================+============+============+=============+==========+==========+================+================+==============+==================+==========+=============+==================+========+=============+================+================+==============+=============+=============+===================+====================+================+============================+=====================+=====================+=====================+=====================+=============+=============+========+============+=============+=====================+=============+===========+==============+=========+=============+===============+==========+==========+============================+======+======+========+===========+===========+============+============+============+===============+=================+================+===============+==============+===========+============+===========+===============+=====================+===================+=============+=======================+==================+============+==============+===============+=================+=====================+====================+==============+==================+===========+===========+=============+============+==========================+=========+==========+======================+======================+======+============+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIM < WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMI T 10; -+---------------------+------------+---------------------------------------------------+-----------+----------------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+----------------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+----------------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+=====================+============+===================================================+===========+============================+============+===========+============+==========+=====================+==============+======+===========+================================================================+=========+=========+===================+=================+===============+=============+=================+==================+=================+============+============+=============+==========+==========+================+================+==============+==================+==========+=============+==================+========+=============+================+================+==============+=============+=============+===================+====================+================+============================+=====================+=====================+=====================+=====================+=============+=============+========+============+=============+=====================+=============+===========+==============+=========+=============+===============+==========+==========+============================+======+======+========+===========+===========+============+============+============+===============+=================+================+===============+==============+===========+============+===========+===============+=====================+===================+=============+=======================+==================+============+==============+===============+=================+=====================+====================+==============+==================+===========+===========+=============+============+==========================+=========+==========+======================+======================+======+============+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT * FROM hits_100m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIM < WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMI T 10; -+---------------------+------------+---------------------------------------------------+-----------+----------------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+----------------------------------------------------------------+---------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+----------------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+-----------+--------------+---------+-------------+---------------+----------+----------+----------------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+--------------------------+---------+----------+----------------------+----------------------+------+------------+ -+=====================+============+===================================================+===========+============================+============+===========+============+==========+=====================+==============+======+===========+================================================================+=========+=========+===================+=================+===============+=============+=================+==================+=================+============+============+=============+==========+==========+================+================+==============+==================+==========+=============+==================+========+=============+================+================+==============+=============+=============+===================+====================+================+============================+=====================+=====================+=====================+=====================+=============+=============+========+============+=============+=====================+=============+===========+==============+=========+=============+===============+==========+==========+============================+======+======+========+===========+===========+============+============+============+===============+=================+================+===============+==============+===========+============+===========+===============+=====================+===================+=============+=======================+==================+============+==============+===============+=================+=====================+====================+==============+==================+===========+===========+=============+============+==========================+=========+==========+======================+======================+======+============+ -write error -sql>\q --- плохой запрос - вынимаем все столбцы.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER B \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER B \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER B \q --- большая сортировка.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER BY SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER B \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER BY SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER B \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER BY SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER B \q --- большая сортировка по строкам.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER B \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER B \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_100m WHERE SearchPhrase not like '' ORDER B \q --- большая сортировка по кортежу.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL not like '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE UR 1 10 0000 ORDER BY l DESC LIMI 100000 ORDER BY l DESC LIMIT  25; -+-----------+---+----+ -+===========+===+====+ -+-----------+---+----+ -0 tuples (4.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL not like '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE UR 1 10 0000 ORDER BY l DESC LIMI 100000 ORDER BY l DESC LIMIT  25; -+-----------+---+----+ -+===========+===+====+ -+-----------+---+----+ -0 tuples (2.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE URL not like '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_100m WHERE UR 1 10 0000 ORDER BY l DESC LIMI 100000 ORDER BY l DESC LIMIT  25; -+-----------+---+----+ -+===========+===+====+ -+-----------+---+----+ -0 tuples (2.5s) -sql>\q --- считаем средние длины URL для крупных счётчиков.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, i POSITION(' <' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/ ' IN SUBSTRING(Referer, P POSITION('/' IN SUBSTRING(Referer, PO SITION('//' IN Referer) + < IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITI \q -more>monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, i POSITION(' <' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/ ' IN SUBSTRING(Referer, P POSITION('/' IN SUBSTRING(Referer, PO SITION('//' IN Referer) + < IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITI \q -more>monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, i POSITION(' <' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/ ' IN SUBSTRING(Referer, P POSITION('/' IN SUBSTRING(Referer, PO SITION('//' IN Referer) + < IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITI \q -more>monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Referer not like '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10 100000 ORDER BY l <'' GROUP BY k HAVING count(*) > 100000 ORDER BY l D ESC LIMIT 25; -syntax error, unexpected AS in: "as" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Referer not like '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10 100000 ORDER BY l <'' GROUP BY k HAVING count(*) > 100000 ORDER BY l D ESC LIMIT 25; -syntax error, unexpected AS in: "as" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Referer not like '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10 100000 ORDER BY l <'' GROUP BY k HAVING count(*) > 100000 ORDER BY l D ESC LIMIT 25; -syntax error, unexpected AS in: "as" -sql>\q --- то же самое, но с разбивкой по доменам.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidt \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidt \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidt \q --- много тупых агрегатных функций.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(Resolut -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(Resolut -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(Resolut -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase not like '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidt \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase not like '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidt \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase not like '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidt \q --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidt \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidt \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidt \q --- то же самое, но ещё и без фильтрации.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC L set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 2 -query: SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC L set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 3 -query: SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC L -sql>\q - -times: 1 -query: SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -Connection terminated -monetdbd is running -spawn mclient -u monetdb -d hits -password: -unsupported hash algorithms: gorithms: gorit -restart server: /etc/init.d/monetdb5-sql restart -Stopping MonetDB SQL server: monetdbd. -monetdbd is already running -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -monetdbd not running. -Starting MonetDB SQL server: monetdbd. - -times: 2 -query: SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -Connection terminated -monetdbd is running -spawn mclient -u monetdb -d hits -password: -unsupported hash algorithms: gorithms: goritb/hits -restart server: /etc/init.d/monetdb5-sql restart -Stopping MonetDB SQL server: monetdbd. -monetdbd is already running -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -monetdbd not running. -Starting MonetDB SQL server: monetdbd. - -times: 3 -query: SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -Connection terminated --- агрегация по URL.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -unsupported hash algorithms: gorithms: goritb/hits -restart server: /etc/init.d/monetdb5-sql restart -Stopping MonetDB SQL server: monetdbd. -monetdbd is already running -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -waiting -monetdbd not running. -Starting MonetDB SQL server: monetdbd. - -times: 1 -query: SELECT 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected sqlINT in: "select 1, url, count(*) from hits_100m group by 1" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected sqlINT in: "select 1, url, count(*) from hits_100m group by 1" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected sqlINT in: "select 1, url, count(*) from hits_100m group by 1" -sql>\q --- агрегация по URL и числу.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_100m group by clientip, clientip -" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_100m group by clientip, clientip -" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_100m group by clientip, clientip -" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL not like '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL not like '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+------+-----------+ -+======+===========+ -+------+-----------+ -1 tuple (5.3s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL not like '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL not like '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+------+-----------+ -+======+===========+ -+------+-----------+ -1 tuple (1.3s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL not like '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL not like '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+------+-----------+ -+======+===========+ -+------+-----------+ -1 tuple (1.4s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title not like '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Ref resh AND Title not like '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+-------+-----------+ -+=======+===========+ -+-------+-----------+ -1 tuple (3.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title not like '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Ref resh AND Title not like '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+-------+-----------+ -+=======+===========+ -+-------+-----------+ -1 tuple (0.9s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title not like '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Ref resh AND Title not like '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+-------+-----------+ -+=======+===========+ -+-------+-----------+ -1 tuple (1.0s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+----------------------------------------------------------------------------------------------------------------------------------------------------------------+------+ -: : view : -: : s : -+================================================================================================================================================================+======+ -: tegoryID=657&search_name=&search_price_from=0&search_price_to=292381¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_25=0¶mRangeTo_25=10¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 0%26price%3D0;8000%26price_from%3D0%26price_to%3D8000%26category[]%3D224 : : -: mest : : -: tegoryID=710&search_name=&search_price_from=0&search_price_to=799053¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000 : : -: 0,2013-07-01:2013-07-09;stat_type=campdate;y1=2013;m1=07;d1=01;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: tegoryID=657&search_name=&search_price_from=0&search_price_to=292381¶m_27_33=yep¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo : : -: _16=5000¶mRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_25=0¶mRangeTo_25=10¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 01:2013-07-14&m2=07&stat_type=custom&onpage=100&group_by=phrase, retargeting&d1=08&target_1=0&cmd=showCampStat&ulogin=kuncevo-ru&reverse=ARRAY(0x17e9c4c0)&tar : : -: get_all=1&online_stat=0&m1=07&cid=4601559&goals=0&offline_stat=0&y2=13 : : -: ype}&pos={position}&key={keyword} : : -: _type_id=1&product_type_1_model_kia[]=Spectra : : -: duct_type_id=1&product_type_1_model_hyundai[]=Sonata : : -: duct_type_id=1&product_type_1_model_hyundai[]=SantaFe : : -: yundai_comp=find_in_set_any&product_type_id=1&product_type_1_model_hyundai[]=Tucson : : -: %3Dpopularity_desc%26num%3D100%26price_from%3D8000%26price_to%3D11000%26price%3D8000;11000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D100%26price_from%3D24000%26price_to%3D27000%26price%3D24000;27000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D20%26price_from%3D15000%26price_to%3D17000%26price%3D15000;17000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D20%26price_from%3D0%26price_to%3D15000%26price%3D0;15000%26category[]%3D224%26data_form%3DY : : -: 01:2013-07-14&m2=07&stat_type=custom&onpage=10000&group_by=phrase&d1=01&target_1=0&cmd=showCampStat&ulogin=UBRiR66&reverse=1&target_all=1&sort=sum&m1=07&onlin : : -: e_stat=0&cid=6715451&goals=0&y2=13&offline_stat=0 : : -: 1,2013-06-01:2013-07-11;ulogin=pb-context;stat_type=campdate;y1=2013;m1=07;d1=11;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: n=Прочитайте истории успеха клие : : -: 13-06-10:2013-07-14&stat_type=custom&group=week&onpage=10000&group_by=phrase, date&d1=08&target_1=0&cmd=showCampStat&ulogin=izosfera2012&reverse=ARRAY(0x19231 : : -: 210)&target_all=1&online_stat=0&m1=07&cid=5791188&goals=0&offline_stat=0&y2=13 : : -: 013-05-31&m2=07&group=week&d1=15&cmd=showCampStat&reverse=1&detail=yes&target_all=yes&sort=sum&online_stat=1&m1=07&cid=6744826&goals=0&y2=13&offline_stat=0 : : -: 013-04-30&m2=07&group=day&d1=15&cmd=showCampStat&ulogin=yar-tehnostroy&detail=yes&target_all=yes&online_stat=1&m1=06&cid=3539794&goals=0&offline_stat=0&y2=13 : : -: 2,2013-07-08:2013-07-14;stat_type=campdate;y1=2013;m1=06;d1=15;y2=2013;m2=07;d2=15;group=day;target_all=1&xls=1 : : -: sc%26num%3D20%26price%3D0;39000%26price_from%3D0%26price_to%3D39000%26category[]%3D1732 : : -: 02:2013-07-14&m2=07&stat_type=custom&group=day&onpage=100&group_by=date&d1=14&target_1=0&cmd=showCampStat&ulogin=aaamotors2009&reverse=ARRAY(0x4029e50)&target : : -: _all=1&online_stat=1&m1=06&cid=5300235&goals=0&offline_stat=0&y2=13 : : -: tegoryID=658&search_name=&search_price_from=0&search_price_to=845243¶m_27_40=yep¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo : : -: _16=5000¶mRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_22=1¶mRangeTo_22=1500 : : -: ,2013-07-01:2013-07-07&m2=07&group=day&d1=08&cmd=showCampStat&ulogin=conrays-mos&phrasedate=yes&online_stat=0&m1=07&cid=5073455&goals=0&offline_stat=0&y2=13 : : -: 4,2013-07-01:2013-07-15;stat_type=campdate;y1=2013;m1=05;d1=01;y2=2013;m2=05;d2=31;group=month;target_all=1&xls=1 : : -: 10:2013-07-14&m2=07&stat_type=custom&group=week&onpage=10000&group_by=phrase, date&d1=08&cmd=showCampStat&target_1=0&ulogin=avtogermes2013&reverse=1&sort=sum& : : -: target_all=1&online_stat=0&m1=07&cid=5767794&goals=0&y2=13&offline_stat=0 : : -: 7,2013-06-24:2013-06-30;stat_type=campdate;y1=2013;m1=07;d1=08;y2=2013;m2=07;d2=14;group=week;target_all=1&xls=1 : : -: 89779;cid=5688769;cid=5846786;cid=5846794;cid=5846798;cid=5846824;cid=5846814;cid=5846776;cid=5689791;cid=5902210;cid=5843727;cid=5945010;cid=7085900;cid=7086 : : -: 434;cid=7086465;cid=7086475;cid=7086483;cid=7086497;cid=7086504;cid=7086506;cid=7086507;cid=7086509;cid=7043324;cid=7085797;cid=7086363;cid=7086379;cid=708639 : : -: 4;cid=7086405;cid=7086505;cid=7086449;cid=7123926;cid=7124057;cid=7123123;cid=7124067;cid=7123939;cid=7124061;cid=7124064;cid=7123937;cid=7124062;cid=7124002; : : -: cid=7124004;cid=7124073;cid=7123987;cid=7123999;cid=7123996;cid=7124025;cid=7124020;cid=7124026;offline_stat=0;online_stat=0;stat_periods=2013-07-12:2013-07-1 : : -: 4,2013-07-01:2013-07-11,2013-07-04:2013-07-11;ulogin=kayak-ru;stat_type=campdate;y1=2013;m1=07;d1=12;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: 0,2013-06-01:2013-06-30;stat_type=campdate;y1=2013;m1=06;d1=01;y2=2013;m2=07;d2=15;group=day;target_all=1&xls=1 : : -: tegoryID=652&search_name=&search_price_from=0&search_price_to=102210¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000¶m_19_23=yep¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: ampStat&target_1=0&reverse=1&sort=clicks&target_all=1&m1=06&online_stat=0&cid=7095075&goals=0&y2=13&offline_stat=0 : : -: ampStat&target_1=0&reverse=1&sort=clicks&target_all=1&online_stat=0&m1=06&cid=7085872&goals=0&y2=13&offline_stat=0 : : -: _type_id=1&product_type_1_model_hyundai[]=Accent&product_type_1_model_hyundai[]=Verna : : -: 0&cmd=showCampStat&reverse=1&sort=clicks&target_all=1&online_stat=0&m1=06&cid=7085610&goals=0&offline_stat=0&y2=13 : : -: tegoryID=648&search_name=ариада&search_price_from=0&search_price_to=2111860¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000& : : -: paramRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 5,2013-06-03:2013-07-14;ulogin=stoli-su;stat_type=campdate;y1=2013;m1=07;d1=08;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -+----------------------------------------------------------------------------------------------------------------------------------------------------------------+------+ -1000 tuples (1.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+----------------------------------------------------------------------------------------------------------------------------------------------------------------+------+ -: : view : -: : s : -+================================================================================================================================================================+======+ -: tegoryID=657&search_name=&search_price_from=0&search_price_to=292381¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_25=0¶mRangeTo_25=10¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 0%26price%3D0;8000%26price_from%3D0%26price_to%3D8000%26category[]%3D224 : : -: mest : : -: tegoryID=710&search_name=&search_price_from=0&search_price_to=799053¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000 : : -: 0,2013-07-01:2013-07-09;stat_type=campdate;y1=2013;m1=07;d1=01;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: tegoryID=657&search_name=&search_price_from=0&search_price_to=292381¶m_27_33=yep¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo : : -: _16=5000¶mRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_25=0¶mRangeTo_25=10¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 01:2013-07-14&m2=07&stat_type=custom&onpage=100&group_by=phrase, retargeting&d1=08&target_1=0&cmd=showCampStat&ulogin=kuncevo-ru&reverse=ARRAY(0x17e9c4c0)&tar : : -: get_all=1&online_stat=0&m1=07&cid=4601559&goals=0&offline_stat=0&y2=13 : : -: ype}&pos={position}&key={keyword} : : -: _type_id=1&product_type_1_model_kia[]=Spectra : : -: duct_type_id=1&product_type_1_model_hyundai[]=Sonata : : -: duct_type_id=1&product_type_1_model_hyundai[]=SantaFe : : -: yundai_comp=find_in_set_any&product_type_id=1&product_type_1_model_hyundai[]=Tucson : : -: %3Dpopularity_desc%26num%3D100%26price_from%3D8000%26price_to%3D11000%26price%3D8000;11000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D100%26price_from%3D24000%26price_to%3D27000%26price%3D24000;27000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D20%26price_from%3D15000%26price_to%3D17000%26price%3D15000;17000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D20%26price_from%3D0%26price_to%3D15000%26price%3D0;15000%26category[]%3D224%26data_form%3DY : : -: 01:2013-07-14&m2=07&stat_type=custom&onpage=10000&group_by=phrase&d1=01&target_1=0&cmd=showCampStat&ulogin=UBRiR66&reverse=1&target_all=1&sort=sum&m1=07&onlin : : -: e_stat=0&cid=6715451&goals=0&y2=13&offline_stat=0 : : -: 1,2013-06-01:2013-07-11;ulogin=pb-context;stat_type=campdate;y1=2013;m1=07;d1=11;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: n=Прочитайте истории успеха клие : : -: 13-06-10:2013-07-14&stat_type=custom&group=week&onpage=10000&group_by=phrase, date&d1=08&target_1=0&cmd=showCampStat&ulogin=izosfera2012&reverse=ARRAY(0x19231 : : -: 210)&target_all=1&online_stat=0&m1=07&cid=5791188&goals=0&offline_stat=0&y2=13 : : -: 013-05-31&m2=07&group=week&d1=15&cmd=showCampStat&reverse=1&detail=yes&target_all=yes&sort=sum&online_stat=1&m1=07&cid=6744826&goals=0&y2=13&offline_stat=0 : : -: 013-04-30&m2=07&group=day&d1=15&cmd=showCampStat&ulogin=yar-tehnostroy&detail=yes&target_all=yes&online_stat=1&m1=06&cid=3539794&goals=0&offline_stat=0&y2=13 : : -: 2,2013-07-08:2013-07-14;stat_type=campdate;y1=2013;m1=06;d1=15;y2=2013;m2=07;d2=15;group=day;target_all=1&xls=1 : : -: sc%26num%3D20%26price%3D0;39000%26price_from%3D0%26price_to%3D39000%26category[]%3D1732 : : -: 02:2013-07-14&m2=07&stat_type=custom&group=day&onpage=100&group_by=date&d1=14&target_1=0&cmd=showCampStat&ulogin=aaamotors2009&reverse=ARRAY(0x4029e50)&target : : -: _all=1&online_stat=1&m1=06&cid=5300235&goals=0&offline_stat=0&y2=13 : : -: tegoryID=658&search_name=&search_price_from=0&search_price_to=845243¶m_27_40=yep¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo : : -: _16=5000¶mRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_22=1¶mRangeTo_22=1500 : : -: ,2013-07-01:2013-07-07&m2=07&group=day&d1=08&cmd=showCampStat&ulogin=conrays-mos&phrasedate=yes&online_stat=0&m1=07&cid=5073455&goals=0&offline_stat=0&y2=13 : : -: 4,2013-07-01:2013-07-15;stat_type=campdate;y1=2013;m1=05;d1=01;y2=2013;m2=05;d2=31;group=month;target_all=1&xls=1 : : -: 10:2013-07-14&m2=07&stat_type=custom&group=week&onpage=10000&group_by=phrase, date&d1=08&cmd=showCampStat&target_1=0&ulogin=avtogermes2013&reverse=1&sort=sum& : : -: target_all=1&online_stat=0&m1=07&cid=5767794&goals=0&y2=13&offline_stat=0 : : -: 7,2013-06-24:2013-06-30;stat_type=campdate;y1=2013;m1=07;d1=08;y2=2013;m2=07;d2=14;group=week;target_all=1&xls=1 : : -: 89779;cid=5688769;cid=5846786;cid=5846794;cid=5846798;cid=5846824;cid=5846814;cid=5846776;cid=5689791;cid=5902210;cid=5843727;cid=5945010;cid=7085900;cid=7086 : : -: 434;cid=7086465;cid=7086475;cid=7086483;cid=7086497;cid=7086504;cid=7086506;cid=7086507;cid=7086509;cid=7043324;cid=7085797;cid=7086363;cid=7086379;cid=708639 : : -: 4;cid=7086405;cid=7086505;cid=7086449;cid=7123926;cid=7124057;cid=7123123;cid=7124067;cid=7123939;cid=7124061;cid=7124064;cid=7123937;cid=7124062;cid=7124002; : : -: cid=7124004;cid=7124073;cid=7123987;cid=7123999;cid=7123996;cid=7124025;cid=7124020;cid=7124026;offline_stat=0;online_stat=0;stat_periods=2013-07-12:2013-07-1 : : -: 4,2013-07-01:2013-07-11,2013-07-04:2013-07-11;ulogin=kayak-ru;stat_type=campdate;y1=2013;m1=07;d1=12;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: 0,2013-06-01:2013-06-30;stat_type=campdate;y1=2013;m1=06;d1=01;y2=2013;m2=07;d2=15;group=day;target_all=1&xls=1 : : -: tegoryID=652&search_name=&search_price_from=0&search_price_to=102210¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000¶m_19_23=yep¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: ampStat&target_1=0&reverse=1&sort=clicks&target_all=1&m1=06&online_stat=0&cid=7095075&goals=0&y2=13&offline_stat=0 : : -: ampStat&target_1=0&reverse=1&sort=clicks&target_all=1&online_stat=0&m1=06&cid=7085872&goals=0&y2=13&offline_stat=0 : : -: _type_id=1&product_type_1_model_hyundai[]=Accent&product_type_1_model_hyundai[]=Verna : : -: 0&cmd=showCampStat&reverse=1&sort=clicks&target_all=1&online_stat=0&m1=06&cid=7085610&goals=0&offline_stat=0&y2=13 : : -: tegoryID=648&search_name=ариада&search_price_from=0&search_price_to=2111860¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000& : : -: paramRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 5,2013-06-03:2013-07-14;ulogin=stoli-su;stat_type=campdate;y1=2013;m1=07;d1=08;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -+----------------------------------------------------------------------------------------------------------------------------------------------------------------+------+ -1000 tuples (1.2s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+----------------------------------------------------------------------------------------------------------------------------------------------------------------+------+ -: : view : -: : s : -+================================================================================================================================================================+======+ -: tegoryID=657&search_name=&search_price_from=0&search_price_to=292381¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_25=0¶mRangeTo_25=10¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 0%26price%3D0;8000%26price_from%3D0%26price_to%3D8000%26category[]%3D224 : : -: mest : : -: tegoryID=710&search_name=&search_price_from=0&search_price_to=799053¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000 : : -: 0,2013-07-01:2013-07-09;stat_type=campdate;y1=2013;m1=07;d1=01;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: tegoryID=657&search_name=&search_price_from=0&search_price_to=292381¶m_27_33=yep¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo : : -: _16=5000¶mRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_25=0¶mRangeTo_25=10¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 01:2013-07-14&m2=07&stat_type=custom&onpage=100&group_by=phrase, retargeting&d1=08&target_1=0&cmd=showCampStat&ulogin=kuncevo-ru&reverse=ARRAY(0x17e9c4c0)&tar : : -: get_all=1&online_stat=0&m1=07&cid=4601559&goals=0&offline_stat=0&y2=13 : : -: ype}&pos={position}&key={keyword} : : -: _type_id=1&product_type_1_model_kia[]=Spectra : : -: duct_type_id=1&product_type_1_model_hyundai[]=Sonata : : -: duct_type_id=1&product_type_1_model_hyundai[]=SantaFe : : -: yundai_comp=find_in_set_any&product_type_id=1&product_type_1_model_hyundai[]=Tucson : : -: %3Dpopularity_desc%26num%3D100%26price_from%3D8000%26price_to%3D11000%26price%3D8000;11000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D100%26price_from%3D24000%26price_to%3D27000%26price%3D24000;27000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D20%26price_from%3D15000%26price_to%3D17000%26price%3D15000;17000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D20%26price_from%3D0%26price_to%3D15000%26price%3D0;15000%26category[]%3D224%26data_form%3DY : : -: 01:2013-07-14&m2=07&stat_type=custom&onpage=10000&group_by=phrase&d1=01&target_1=0&cmd=showCampStat&ulogin=UBRiR66&reverse=1&target_all=1&sort=sum&m1=07&onlin : : -: e_stat=0&cid=6715451&goals=0&y2=13&offline_stat=0 : : -: 1,2013-06-01:2013-07-11;ulogin=pb-context;stat_type=campdate;y1=2013;m1=07;d1=11;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: n=Прочитайте истории успеха клие : : -: 13-06-10:2013-07-14&stat_type=custom&group=week&onpage=10000&group_by=phrase, date&d1=08&target_1=0&cmd=showCampStat&ulogin=izosfera2012&reverse=ARRAY(0x19231 : : -: 210)&target_all=1&online_stat=0&m1=07&cid=5791188&goals=0&offline_stat=0&y2=13 : : -: 013-05-31&m2=07&group=week&d1=15&cmd=showCampStat&reverse=1&detail=yes&target_all=yes&sort=sum&online_stat=1&m1=07&cid=6744826&goals=0&y2=13&offline_stat=0 : : -: 013-04-30&m2=07&group=day&d1=15&cmd=showCampStat&ulogin=yar-tehnostroy&detail=yes&target_all=yes&online_stat=1&m1=06&cid=3539794&goals=0&offline_stat=0&y2=13 : : -: 2,2013-07-08:2013-07-14;stat_type=campdate;y1=2013;m1=06;d1=15;y2=2013;m2=07;d2=15;group=day;target_all=1&xls=1 : : -: sc%26num%3D20%26price%3D0;39000%26price_from%3D0%26price_to%3D39000%26category[]%3D1732 : : -: 02:2013-07-14&m2=07&stat_type=custom&group=day&onpage=100&group_by=date&d1=14&target_1=0&cmd=showCampStat&ulogin=aaamotors2009&reverse=ARRAY(0x4029e50)&target : : -: _all=1&online_stat=1&m1=06&cid=5300235&goals=0&offline_stat=0&y2=13 : : -: tegoryID=658&search_name=&search_price_from=0&search_price_to=845243¶m_27_40=yep¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo : : -: _16=5000¶mRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_22=1¶mRangeTo_22=1500 : : -: ,2013-07-01:2013-07-07&m2=07&group=day&d1=08&cmd=showCampStat&ulogin=conrays-mos&phrasedate=yes&online_stat=0&m1=07&cid=5073455&goals=0&offline_stat=0&y2=13 : : -: 4,2013-07-01:2013-07-15;stat_type=campdate;y1=2013;m1=05;d1=01;y2=2013;m2=05;d2=31;group=month;target_all=1&xls=1 : : -: 10:2013-07-14&m2=07&stat_type=custom&group=week&onpage=10000&group_by=phrase, date&d1=08&cmd=showCampStat&target_1=0&ulogin=avtogermes2013&reverse=1&sort=sum& : : -: target_all=1&online_stat=0&m1=07&cid=5767794&goals=0&y2=13&offline_stat=0 : : -: 7,2013-06-24:2013-06-30;stat_type=campdate;y1=2013;m1=07;d1=08;y2=2013;m2=07;d2=14;group=week;target_all=1&xls=1 : : -: 89779;cid=5688769;cid=5846786;cid=5846794;cid=5846798;cid=5846824;cid=5846814;cid=5846776;cid=5689791;cid=5902210;cid=5843727;cid=5945010;cid=7085900;cid=7086 : : -: 434;cid=7086465;cid=7086475;cid=7086483;cid=7086497;cid=7086504;cid=7086506;cid=7086507;cid=7086509;cid=7043324;cid=7085797;cid=7086363;cid=7086379;cid=708639 : : -: 4;cid=7086405;cid=7086505;cid=7086449;cid=7123926;cid=7124057;cid=7123123;cid=7124067;cid=7123939;cid=7124061;cid=7124064;cid=7123937;cid=7124062;cid=7124002; : : -: cid=7124004;cid=7124073;cid=7123987;cid=7123999;cid=7123996;cid=7124025;cid=7124020;cid=7124026;offline_stat=0;online_stat=0;stat_periods=2013-07-12:2013-07-1 : : -: 4,2013-07-01:2013-07-11,2013-07-04:2013-07-11;ulogin=kayak-ru;stat_type=campdate;y1=2013;m1=07;d1=12;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: 0,2013-06-01:2013-06-30;stat_type=campdate;y1=2013;m1=06;d1=01;y2=2013;m2=07;d2=15;group=day;target_all=1&xls=1 : : -: tegoryID=652&search_name=&search_price_from=0&search_price_to=102210¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000¶m_19_23=yep¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: ampStat&target_1=0&reverse=1&sort=clicks&target_all=1&m1=06&online_stat=0&cid=7095075&goals=0&y2=13&offline_stat=0 : : -: ampStat&target_1=0&reverse=1&sort=clicks&target_all=1&online_stat=0&m1=06&cid=7085872&goals=0&y2=13&offline_stat=0 : : -: _type_id=1&product_type_1_model_hyundai[]=Accent&product_type_1_model_hyundai[]=Verna : : -: 0&cmd=showCampStat&reverse=1&sort=clicks&target_all=1&online_stat=0&m1=06&cid=7085610&goals=0&offline_stat=0&y2=13 : : -: tegoryID=648&search_name=ариада&search_price_from=0&search_price_to=2111860¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000& : : -: paramRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 5,2013-06-03:2013-07-14;ulogin=stoli-su;stat_type=campdate;y1=2013;m1=07;d1=08;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -+----------------------------------------------------------------------------------------------------------------------------------------------------------------+------+ -1000 tuples (1.3s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageVi ews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -: icso : chen : ngin : : : iews : -: urce : gine : eid : : : : -: id : id : : : : : -+======+======+======+====================================================================+=====================================================================+=======+ -: : : : : ype=campdate : : -: : : : type=campdate : : : -: : : : : &payment_type=1 : : -: : : : : d=7 : : -: : : : : &mediaType=text : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0 : : : -: : : : : 79.0&authredirlevel=1373600157.0&authredirlevel=1373081665.0&authre : : -: : : : : dirlevel=1372822331.0&authredirlevel=1372563095.0&authredirlevel=13 : : -: : : : : 72303801.0&authredirlevel=1372044532.0&authredirlevel=1371730939.0& : : -: : : : : authredirlevel=1371471570.0&authredirlevel=1371212358.0&cmd=showCam : : -: : : : : p&cid=2856748 : : -: : : : t : : : -: : : : : ype=campdate : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : -: : : : : s=1 : : -: : : : : owCamps : : -: : : : 1&mediaType=text : : : -: : : : : dirlevel=1373731047.0&cmd=showCamps : : -: : : : : 09.0&authredirlevel=1370955451.0&cmd=showCamp&cid=5477450 : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : type=campdate : : : -: : : : : d=7 : : -: : : : : 47.0&cmd=showCamp&cid=4567586 : : -: : : : : 6 : : -: : : : : OqWshMF.pl?cmd=showCamps&emedia_camps=0&ulogin=lena-san-std : : -: : : : : n= : : -: : : : : 82.0&cmd=showCamp&cid=2753896 : : -: : : : : 5 : : -: : : : 75 : : : -: : : : : &mediaType=text : : -: : : : : 0 : : -: : : : : 2 : : -: : : : 19 : : : -: : : : 93 : : : -: : : : : 9 : : -: : : : : dirlevel=1373869482.0&cmd=showCamp&cid=2997085 : : -: : : : : 3 : : -: : : : 21 : : : -: : : : : 88.0 : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 01.0&cmd=showCamp&cid=4567586 : : -: : : : howCampStat&cid=7056273&offline_stat=0&online_stat=1&stat_periods= : 56273&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y2= : : -: : : : 2013-05-25:2013-06-24,2013-05-22:2013-05-23,2013-05-17:2013-05-18& : 13&group=day : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : : -: : : : &goals=0&target_all=yes : : : -: : : : ks=1 : : : -: : : : : 1 : : -: : : : : 1 : : -: : : : : 7 : : -: : : : =total&target_all=1&page=1&cmd=showCampStat&cid=3016175&show_banne : owCampStat&cid=3016175&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : rs_stat=1 : 12-02-03:2012-02-04,2011-08-01:2011-09-04,2010-12-29:2011-01-02&typ : : -: : : : : es=total&page=&group=day&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&sh : : -: : : : : ow_banners_stat=1 : : -: : : : 37 : : : -: : : : 65 : : : -: : : : 01 : : : -: : : : : 5 : : -: : : : : 9 : : -: : : : 19 : : : -: : : : 71 : : : -: : : : : 1 : : -: : : : 08 : : : -: : : : 81 : : : -: : : : : 1 : : -: : : : 45 : : : -: : : : : 0 : : -: : : : : 5 : : -: : : : : 8 : : -: : : : type=campdate : : : -: : : : 90 : gn_direct_plastik_den : : -: : : : : 3 : : -: : : : 40 : : : -: : : : 39 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 03 : : : -: : : : : 6 : : -: : : : ks=1 : : : -: : : : : 7 : : -: : : : : 2 : : -: : : : howCamp&cid=5940082 : : : -: : : : : 9 : : -: : : : 7-.31620-2.1373861604902917.198343986.198343986..64036780.1406.9b1 : 7 : : -: : : : : 1 : : -: : : : : Informers.html : : -: : : : d : : : -: : : : 29 : : : -: : : : 95 : : : -: : : : ransfer : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6870158 : : -: : : : 68 : : : -: : : : : dirlevel=1342185655.0&cmd=showCamps&ulogin=beautyland-artha8 : : -: : : : : 1 : : -: : : : : 8 : : -: : : : 71 : : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 4 : : -: : : : : 018058&cmd=showCamp : : -: : : : 76 : owCampStat&cid=5109476&detail=Yes&types=days : : -: : : : : 9 : : -: : : : : owSubClientCamps : : -: : : : : 1 : : -: : : : : 0 : : -: : : : type=campdate : : : -: : : : 91 : : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=4718384 : : -: : : : : w.advertising.yandex.ru/welcome/pdf/direct_booklet.pdf : : -: : : : : =Yes&types=days&cid=7114664 : : -: : : : : 7 : : -: : : : : &mediaType=text : : -: : : : : 51.0 : : -: : : : ditBannerEasy&cid=7080496&bid=188854799 : 080496&cmd=showCamp : : -: : : : : 8 : : -: : : : 56 : owCampStat&cid=5895656&detail=Yes&types=days : : -: : : : edirlevel=1373786750.0&cmd=showCampStat&cid=6421801&cid=6424294&ci : : : -: : : : d=6487283&cid=6493829&cid=6494462&cid=6495189&stat_type=campdate&c : : : -: : : : srf_token=CzqM8_OaIwxa28-a : : : -: : : : 28 : : : -: : : : : 6 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : type=campdate : : : -: : : : 77 : lshoj-kvartiry : : -: : : : 83 : : : -: : : : : vancedForecast : : -: : : : howCamp&cid=2925555 : owCamp&cid=2925555 : : -: : : : howCamps&ulogin=aplarru : 9&ulogin=aplarru : : -: : : : : 1 : : -: : : : 16 : : : -: : : : : ger : : -: : : : : 52.0&authredirlevel=1373884890.0 : : -: : : : l=Yes&types=days&cid=7026863 : : : -: : : : : 4 : : -: : : : : 5 : : -: : : : 43 : : : -: : : : : 1 : : -: : : : : 9 : : -: : : : : owCamp&cid=7129447#193577203 : : -: : : : 38 : 46.0 : : -: : : : : 8 : : -: : : : 44 : : : -: : : : : 8 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6486629 : : -: : : : : 65.0 : : -: : : : : =Yes&types=days&cid=7131552 : : -: : : : : 3 : : -: : : : howCamp&cid=7129026&tab=wait : owCamp&cid=7129026&tab=active : : -: : : : : 3 : : -: : : : : =Yes&types=days&cid=7015594 : : -: : : : 48 : : : -: : : : : 5 : : -: : : : : 3 : : -: : : : 78 : : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=5826713 : : : -: : : : : 5 : : -: : : : : 1 : : -: : : : : =Yes&types=days&cid=5309194 : : -: : : : howCamps : owCamps : : -: : : : : 6 : : -: : : : 76 : : : -: : : : 21 : : : -: : : : 55 : : : -: : : : : 5 : : -: : : : : 1 : : -: : : : : 8 : : -: : : : : 6 : : -: : : : : 6 : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : : -: : : : : type=campdate : : -: : : : t : : : -: : : : l=Yes&types=days&cid=6752443 : 52443&phrasedate=Yes&target_all=1 : : -: : : : howCamp&cid=6501536&tab=&page=2 : : : -: : : : : 8 : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=7026863 : : -: : : : 41 : : : -: : : : : 5 : : -: : : : : owCamps&ulogin=super-svictor : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6411871 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6189242 : : -: : : : l=Yes&cid=6009684&showCampStat=cmd&types=days&target_all=1&d1=08&m : owCampStat&cid=6009684&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : 1=07&y1=13&d2=08&m2=07&y2=13&group=day : 13-02-25:2013-02-25,2013-02-26:2013-02-26,&detail=yes&page=&y1=2013 : : -: : : : : &m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : 1&mediaType=text : : : -: : : : : =Yes&types=days&cid=6752443 : : -: : : : 42 : : : -: : : : : 3 : : -: : : : howCamps : 2 : : -: : : : 263.0&cmd=showCamp&cid=5830159 : owCampStat&cid=5830159&detail=Yes&types=days : : -: : : : : 7 : : -: : : : : =Yes&types=days&cid=6467205 : : -: : : : : =Yes&types=days&cid=6467203 : : -: : : : 89 : : : -: : : : howCamp&cid=5088603 : owCamp&cid=5088603 : : -: : : : 77 : : : -: : : : : 6 : : -: : : : : 78.0 : : -: : : : 42 : : : -: : : : 38 : owCampStat&cid=6356238&detail=Yes&types=days : : -: : : : howCamps : 6 : : -: : : : : 9 : : -: : : : howCamps&ulogin=discoverycruise : =Yes&types=days&cid=7065506&ulogin=discoverycruise : : -: : : : : 33.0&cmd=showCampStat&detail=Yes&types=days&cid=6393197 : : -: : : : : 7 : : -: : : : ditCamp&cid=5309194 : =Yes;types=days;cid=5309194 : : -: : : : : owCamp&cid=6980076 : : -: : : : : dirlevel=1373863808.0&authredirlevel=1373522397.0&cmd=showCamp&cid= : : -: : : : : 6051926&tab=active : : -: : : : : =easy&from=metrika : : -: : : : : =Yes&types=days&cid=7135716 : : -: : : : : 3 : : -: : : : howCamp&cid=7129447&tab=wait : owCamp&cid=7129447&tab=active : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : 5279956&tab=media : owCamp&cid=5279956&tab=active : : -: : : : ditBannerEasy&cid=6835814&bid=171781249 : 835814&cmd=showCamp : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6931378 : : -: : : : : ype=campdate : : -: : : : 68 : : : -: : : : 42 : : : -: : : : l=Yes&types=days&cid=4718384 : : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 7 : : -: : : : : owCamps&ulogin=visacity08 : : -: : : : : 109741&cmd=showCamp : : -: : : : type=campdate&&ulogin=vnukovo-piter : owCamps&ulogin=vnukovo-piter : : -: : : : : 1 : : -: : : : 3243479&tab=media : owCamp&cid=3243479&tab=active : : -: : : : : 1 : : -: : : : ditBannerEasy&cid=7123828&bid=193277936 : 123828&cmd=showCamp : : -: : : : : 6 : : -: : : : : 0 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : howCamp&cid=7133264 : owCamp&cid=7133264&tab=wait : : -: : : : : 5 : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : howCamps&ulogin=well-mir-ru : 0&ulogin=well-mir-ru : : -: : : : 66 : : : -: : : : 83 : : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : 5 : : -: : : : : 4 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : : owCamps&ulogin=modul-sdskbeton : : -: : : : : =Yes&types=days&cid=7064245 : : -: : : : &ulogin= : : : -: : : : : 3 : : -: : : : howCampStat&cid=3370690&detail=Yes&types=days : 0 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : : 3 : : -: : : : : owCamps : : -: : : : : 7 : : -: : : : : come=yes : : -: : : : : 3 : : -: : : : : 9 : : -: : : : : 3 : : -: : : : : 6 : : -: : : : howCampStat&cid=7109552&detail=Yes&types=days : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7109552&sh : : -: : : : : ow_banners_stat=1 : : -: : : : howCamp&cid=7134723 : : : -: : : : howCamps&ulogin=alfenjoy-dir : 3&ulogin=alfenjoy-dir : : -: : : : howCamps&ulogin=allstarshop-yd : owCamps&ulogin=allstarshop-yd : : -: : : : howCamps : 7 : : -: : : : : 6 : : -: : : : 35 : : : -: : : : 96&ulogin=kuncevo-ru : vo-avto-obyavlyaet-start-prodazh-LEXUS-ES-i-LEXUS-RX-spetsialnyh-ve : : -: : : : : rsiy.tmex : : -: : : : : =Yes&types=days&cid=6144446 : : -: : : : : 2 : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 137413&cmd=orderCamp&agree=yes : : : -: : : : : 8 : : -: : : : 80 : : : -: : : : 994219&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y : 9 : : -: : : : 2=13&group=day : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=3346173 : : -: : : : : =Yes&types=days&cid=4300226 : : -: : : : 90 : : : -: : : : id=7 : : : -: : : : : owCampStat&cid=4194803&cid=5256554&cid=5718709&cid=6790267&cid=6790 : : -: : : : : 271&cid=7070376&stat_type=campdate&ulogin=mybalcony-ru&csrf_token=L : : -: : : : : BcSJLrXLKh9zXYq : : -: : : : 38 : owCampStat&cid=6356238&detail=Yes&types=days : : -: : : : 431468&phrasedate=Yes&target_all=1 : owCampStat&cid=5431468&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-06-27:2013-07-10,2013-01-01:2013-06-01,2013-06-01:2013-07-03&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 3 : : -: : : : 74 : : : -: : : : all : : : -: : : : : 7 : : -: : : : : 0 : : -: : : : 16 : : : -: : : : : owClients : : -: : : : : 6 : : -: : : : : =Yes&types=days&cid=1475303 : : -: : : : : 2 : : -: : : : l=Yes&types=days&cid=6942653 : owCampStat&cid=6942653&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-06-22:2013-06-22,2013-06-16:2013-06-18,2013-06-17:2013-06-18&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : 97 : : : -: : : : howCamps : 6 : : -: : : : 51 : : : -: : : : : 480202&cmd=showCamp : : -: : : : howCamp&cid=3973331&ulogin=kuncevo-ru : : : -: : : : : owCamps&ulogin=visamaster08 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=6800439 : : : -: : : : 09 : owCamp&cid=6797009 : : -: : : : : 9 : : -: : : : : dirlevel=1373859813.0&cmd=showCampStat&cid=5538008&detail=Yes&types : : -: : : : : =days : : -: : : : 316.0&cmd=showCamp&cid=5993938 : : : -: : : : : 2 : : -: : : : : 7 : : -: : : : : =Yes&types=days&cid=7103125 : : -: : : : : 8 : : -: : : : howCamp&cid=7129447&tab=wait : owCamp&cid=7129447&tab=active : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6883763 : : -: : : : : 7 : : -: : : : : ype=campdate : : -: : : : : =Yes&types=days&cid=1724608 : : -: : : : : 5 : : -: : : : : 2 : : -: : : : : 3 : : -: : : : : 2 : : -: : : : 09 : : : -: : : : : 4 : : -: : : : : ed : : -: : : : = : 17.0&authredirlevel=1373534053.0&cmd=apiApplicationList&ulogin= : : -: : : : : 2 : : -: : : : HB.31620-2.1373870178024287.199046816.199046816..14150262.1406.c65 : : : -: : : : &rnd=0.9338765616586568 : : : -: : : : 368.0&cmd=showCamp&cid=4685945 : 74.0&authredirlevel=1363929368.0&cmd=showCamp&cid=4685945 : : -: : : : : 5 : : -: : : : : 97.0&cmd=ForecastByWords : : -: : : : 71 : : : -: : : : howCampStat&cid=2196876&detail=Yes&types=days : 6 : : -: : : : = : : : -: : : : howCampStat&cid=7056273&offline_stat=0&online_stat=1&stat_periods= : 56273&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y2= : : -: : : : 2013-05-25:2013-06-24,2013-05-22:2013-05-23,2013-05-17:2013-05-18& : 13&group=day : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : : -: : : : &goals=0&target_all=yes : : : -: : : : : 9 : : -: : : : edirlevel=1373868161.0&&cid=7027214&cmd=showCamp : owCampStat&cid=7027214&detail=Yes&types=days : : -: : : : : =Yes&types=days&cid=6630335 : : -: : : : : owCamps : : -: : : : nt_country=225&cmd=editBannerEasy¤cy=YND_FIXED&interface_typ : 136912&cmd=showCamp&unglued=194242390 : : -: : : : e=easy¬new=yes : : : -: : : : : 7 : : -: : : : : 6 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-11:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-03:201 : : -: : : : 013-07-15,2013-07-01:2013-07-14,2013-07-05:2013-07-13&stat_type=ca : 3-07-15,2013-07-11:2013-07-15,2013-07-01:2013-07-14&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&target_al : ate&y1=2013&m1=07&d1=03&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : l=1 : : : -: : : : : 83.0 : : -: : : : howCamps&ulogin=nordex-grandit : 57.0&cmd=showCampStat&detail=Yes&types=days&cid=6947313&ulogin=nord : : -: : : : : ex-grandit : : -: : : : : =Yes&types=days&cid=7068093 : : -: : : : : =Yes&types=days&cid=6834748 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2011-10-27:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2011-10-27:201 : : -: : : : 011-10-27,2011-08-31:2011-08-31,2011-08-26:2011-08-26&stat_type=ca : 1-10-27,2011-08-31:2011-08-31,2011-08-26:2011-08-26&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=14&y2=2013&m2=07&d2=14&group=day&target_al : ate&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : l=1 : : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 38 : 07.0&authredirlevel=1373876446.0 : : -: : : : : 134782&cmd=showCamp&fixated=150055634 : : -: : : : : dirlevel=1364366494.0&cmd=showCamp&cid=6106774 : : -: : : : : =Yes&types=days&cid=7072622 : : -: : : : : owClients : : -: : : : : 113103&cmd=showCamp : : -: : : : : 2 : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : : 5 : : -: : : : : 05.6&authredirlevel=1372014905.5&authredirlevel=1372014905.4&authre : : -: : : : : dirlevel=1372014905.3&authredirlevel=1372014905.2&authredirlevel=13 : : -: : : : : 72014905.1&authredirlevel=1372014905.0&authredirlevel=1370480737.6& : : -: : : : : authredirlevel=1370480737.5&authredirlevel=1370480737.4&authredirle : : -: : : : : vel=1370480737.3&authredirlevel=1370480737.2&authredirlevel=1370480 : : -: : : : : 737.1&authredirlevel=1370480737.0&cmd=showCampStat&stat_type=campda : : -: : : : : te&target_0=1&target_1=1&target_all=1 : : -: : : : : ulogin= : : -: : : : : 5 : : -: : : : howCamp&cid=6980076 : owCampStat&cid=6980076&detail=Yes&types=days : : -: : : : &ulogin= : ulogin= : : -: : : : howClients : owCamps&ulogin=K-PV-beeline : : -: : : : l=Yes&types=days&cid=88689 : : : -: : : : 32 : : : -: : : : 15 : owCampStat&cid=5991615&detail=Yes&types=days : : -: : : : howCamp&cid=6980076 : owCamp&cid=6980076 : : -: : : : : =Yes&types=days&cid=5871623 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 8 : : -: : : : : owCampStat&cid=4917291&detail=Yes&types=days : : -: : : : : 2 : : -: : : : : 90.0 : : -: : : : : 8 : : -: : : : : owCamps&ulogin=domusmebel-ru : : -: : : : : 0 : : -: : : : : =Yes&types=days&cid=4735372 : : -: : : : : 9 : : -: : : : 49 : : : -: : : : : 5 : : -: : : : 08 : : : -: : : : howCampStat&cid=3353427&offline_stat=0&online_stat=0&stat_periods= : d2=15&target_0=0&stat_periods=2013-07-01:2013-07-06,2013-06-27:2013 : : -: : : : 2013-07-01:2013-07-06,2013-06-27:2013-07-05,2013-06-01:2013-06-30& : -07-05,2013-06-01:2013-06-30&m2=07&stat_type=custom&onpage=100&grou : : -: : : : stat_type=custom&sort=&reverse=ARRAY(0x1639e200)&group_by=tag&filt : p_by=tag, banner, page, phrase&d1=15&target_1=0&cmd=showCampStat&ta : : -: : : : er_tag=&group_by=banner&filter_banner=&group_by=page&filter_page=& : rget_all=1&online_stat=0&m1=07&cid=3353427&goals=0&offline_stat=0&y : : -: : : : filter_page_target=&group_by=phrase&filter_phrase=&goals=0&y1=2013 : 2=13&sort=clicks&reverse=1 : : -: : : : &m1=07&d1=15&y2=2013&m2=07&d2=15&onpage=100 : : : -: : : : : owCamps : : -: : : : : 2 : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 8 : : -: : : : 44 : owCampStat&cid=4154744&detail=Yes&types=days : : -: : : : 01 : : : -: : : : site_promotion_search : : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 02.0 : : -: : : : : 7 : : -: : : : howCamp&cid=6522990&tab=off : owCamp&cid=6522990 : : -: : : : : 9 : : -: : : : : 8 : : -: : : : : 9 : : -: : : : : 5 : : -: : : : : 4 : : -: : : : 813.0&cmd=showCamp&cid=5210661 : : : -: : : : : 63.0&cmd=showCamp&cid=5830159 : : -: : : : : =Yes&types=days&cid=6993362 : : -: : : : : vancedForecast : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : 7090667&cmd=showCamp : owCampStat&cid=7090667&detail=Yes&types=days : : -: : : : 77&ulogin=svk-direct : : : -: : : : : 3 : : -: : : : : 1 : : -: : : : howCampStat&cid=5975585&detail=Yes&types=days : 5 : : -: : : : : 8 : : -: : : : : 1 : : -: : : : : 41.0&authredirlevel=1373879619.0&cmd=showCamp&cid=5356346 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-03-01:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-03-01:201 : : -: : : : 013-07-15,2013-06-01:2013-06-14,2013-06-01:2013-07-15&stat_type=ca : 3-07-15,2013-06-01:2013-06-14,2013-06-01:2013-07-15&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=month&target_ : ate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : all=1 : : : -: : : : : owCamps&ulogin=infoprom51 : : -: : : : : 64.0&cmd=showCamp&cid=5028648 : : -: : : : 64 : : : -: : : : : 6 : : -: : : : : 34.0&cmd=ForecastByWords : : -: : : : : 4 : : -: : : : ts : : : -: : : : : 2 : : -: : : : ncy : : : -: : : : : 8 : : -: : : : : 55.0&cmd=showCampStat&detail=Yes&types=days&cid=6849113 : : -: : : : howCamps&ulogin=aplarru : 9&ulogin=aplarru : : -: : : : : =Yes&types=days&cid=5039376 : : -: : : : : owCamp&cid=4748947&ulogin=cont5nn#71488518 : : -: : : : _camps=0&cmd=showSubClientCamps&authredirlevel=1373873308.0&settin : dirlevel=1373880608.0&media_camps=0&cmd=showSubClientCamps&authredi : : -: : : : gs_sorted=&sort=total&reverse=1 : rlevel=1373873308.0&settings_sorted=&sort=total&reverse=0 : : -: : : : howCamp&cid=4581573&tab=decline : owCamp&cid=4581573&tab=active : : -: : : : : 9 : : -: : : : howClients : vancedForecast : : -: : : : : 983763&cmd=showCamp&unglued=194305392 : : -: : : : : 9 : : -: : : : : 5 : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=7030009 : : -: : : : 0 : : : -: : : : : 8 : : -: : : : : 9 : : -: : : : : 65.0&cmd=showCamp&cid=7135509 : : -: : : : 642.0 : ype=campdate : : -: : : : : 5 : : -: : : : 35 : 5 : : -: : : : : dirlevel=1373877895.0&cmd=showCamp&cid=3055906 : : -: : : : : 6 : : -: : : : 099442&cid=7100140&cid=7100144&cid=7100148&cid=7100150&cid=7132172 : owCampStat&cid=7099442&cid=7100140&cid=7100144&cid=7100148&cid=7100 : : -: : : : &cid=7132174&cid=7132175&cid=7132177&cid=7132179&cid=7132632&cid=7 : 150&cid=7132172&cid=7132174&cid=7132175&cid=7132177&cid=7132179&cid : : -: : : : 132838&cid=7132888&cid=7133059&cid=7133217&stat_type=campdate&csrf : =7132632&cid=7132838&cid=7132888&cid=7133059&cid=7133217&offline_st : : -: : : : _token=aENmaxzslTI5oMqD : at=0&online_stat=0&stat_periods=2013-07-01:2013-07-13,2013-07-01:20 : : -: : : : : 13-07-12,2013-06-14:2013-07-11&stat_type=campdate&y1=2013&m1=07&d1= : : -: : : : : 15&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : 35 : 63.0&authredirlevel=1373873632.0 : : -: : : : : 1 : : -: : : : 73 : : : -: : : : howCampStat&cid=6981382&offline_stat=0&online_stat=1&stat_periods= : owCampStat&cid=6981382&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : 2013-06-10:2013-06-25,2013-06-10:2013-06-24,&detail=yes&page=&y1=2 : 13-06-10:2013-06-25,2013-06-10:2013-06-24,&detail=yes&page=&y1=2013 : : -: : : : 013&m1=07&d1=05&y2=2013&m2=07&d2=12&group=day&goals=0&target_all=y : &m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : es : : : -: : : : : =Yes&types=days&cid=6621650 : : -: : : : gSearch&who=camps&cid=&camp_login=&camp_name=атлант : 4&ulogin=atlant-b2bnew : : -: : : : : 0 : : -: : : : : owCamps&ulogin=domusmebel-ru : : -: : : : : 3 : : -: : : : : owCamp&cid=3601040 : : -: : : : : =Yes&types=days&cid=6858643 : : -: : : : : =Yes&types=days&cid=7109462 : : -: : : : : =Yes&types=days&cid=7085888 : : -: : : : 25 : : : -: : : : : owCamps : : -: : : : d=18770681&resp_desc=success : owCamps : : -: : : : howCamp&cid=6980076 : owCampStat&cid=6980076&detail=Yes&types=days : : -: : : : : 5 : : -: : : : : 4 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=5129796 : : -: : : : : 7 : : -: : : : howCampStat&cid=7104084&detail=Yes&types=days : 104084&cmd=showCamp : : -: : : : : 6 : : -: : : : : 2 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=3427083 : : : -: : : : : =Yes&types=days&cid=7060114 : : -: : : : l=Yes&types=days&cid=6833696 : owCampStat&cid=6833696&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-20:2013-06-21,&detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m : : -: : : : : 2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : : 137972&cmd=showCamp&unglued=194336987 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : &mediaType=text&firstbanner=std : : -: : : : 137830&cmd=orderCamp&agree=yes : : : -: : : : : 9 : : -: : : : 09 : : : -: : : : : 131075&cmd=showCamp : : -: : : : : 2 : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6960697 : : -: : : : 7129022&cmd=showCamp : owCampStat&cid=7129022&detail=Yes&types=days : : -: : : : 43 : : : -: : : : : 8 : : -: : : : 17 : owCamp&cid=7082517 : : -: : : : : 6 : : -: : : : 31&ulogin=kuncevo-ru : : : -: : : : 394451&phrasedate=Yes&target_all=1 : owCampStat&cid=4394451&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 12-01-01:2012-11-21,2012-01-01:2012-07-31,2012-03-23:2012-03-23&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 1 : : -: : : : howCamps : owCamps : : -: : : : : 1 : : -: : : : : 6 : : -: : : : gSearch&who=camps&cid=&camp_login=&camp_name=дизв : 9&ulogin=dizvitr-b2b : : -: : : : : 7 : : -: : : : : 9 : : -: : : : : campdate : : -: : : : : 0 : : -: : : : : 6 : : -: : : : : 6 : : -: : : : howCamps&ulogin=oknabuduschego : =Yes&types=days&cid=5506368&ulogin=oknabuduschego : : -: : : : howClients&sort=total&reverse=0 : owCamps&ulogin=Rusokon2007 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=4879898 : : -: : : : 1&mediaType=text : : : -: : : : 85 : : : -: : : : howCamps&ulogin=visacity08 : 5&ulogin=visacity08 : : -: : : : 56 : : : -: : : : : owCamps&ulogin=dc12a : : -: : : : 59 : : : -: : : : howCamps&ulogin=ul-smilestar-su : =ul-smilestar-su#list : : -: : : : howCampStat&cid=7053221&detail=Yes&types=days : owCampStat&cid=7053221&offline_stat=0&online_stat=0&stat_periods=&d : : -: : : : : etail=yes&page=&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&g : : -: : : : : oals=0&target_all=yes&show_banners_stat=1 : : -: : : : : =Yes&types=days&cid=7133318 : : -: : : : : 5 : : -: : : : 25 : : : -: : : : howCamp&tab=all&cid=7126549&ulogin=transkontinental2014#194270034 : : : -: : : : : 2 : : -: : : : 02 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 07 : : : -: : : : : 1 : : -: : : : : 9 : : -: : : : howCamp&cid=7078898&tab=off : owCamp&cid=7078898 : : -: : : : 01 : : : -: : : : : 4 : : -: : : : : 35.0 : : -: : : : : owCampStat&cid=2886087&detail=Yes&types=days : : -: : : : : 8 : : -: : : : type=campdate : dirlevel=1373865822.0&cmd=showCampStat&offline_stat=0&online_stat=0 : : -: : : : : &stat_periods=2013-07-04:2013-07-05,2013-06-10:2013-06-13,2013-05-2 : : -: : : : : 1:2013-05-31&stat_type=campdate&y1=2013&m1=07&d1=15&y2=2013&m2=07&d : : -: : : : : 2=15&group=day&target_all=1 : : -: : : : : 0 : : -: : : : : owCamp&cid=6354573#194189392 : : -: : : : : owCamps : : -: : : : 25 : : : -: : : : : 8 : : -: : : : : 04.0&cmd=showCamp&cid=7129428 : : -: : : : howCamp&cid=7130323 : ids=193775525&cid=7130323 : : -: : : : howCamps&ulogin=richton-studio : 7&ulogin=richton-studio : : -: : : : : owCampStat&cid=7034406&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-07-11:2013-07-11,2013-07-12:2013-07-12,2013-07-13:2013-07-13&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 0 : : -: : : : howCamp&tab=decline&cid=6886514 : owCamp&cid=6886514&tab=active : : -: : : : : 33.0 : : -: : : : : 60.0 : : -: : : : : 5 : : -: : : : : dirlevel=1373866480.0&cmd=showCampStat&stat_type=by_agency_clients : : -: : : : : 9 : : -: : : : : 463697&cmd=showCamp : : -: : : : 28 : : : -: : : : : 6 : : -: : : : : 8 : : -: : : : : 4 : : -: : : : 60 : : : -: : : : : 5 : : -: : : : : 7 : : -: : : : : 1 : : -: : : : howCampStat&cid=6456112&offline_stat=0&online_stat=0&stat_periods= : owCampStat&cid=6456112&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : 2013-07-10:2013-07-10,2013-07-09:2013-07-09,2013-07-08:2013-07-08& : 13-07-10:2013-07-10,2013-07-09:2013-07-09,2013-07-08:2013-07-08&phr : : -: : : : phrasedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : =day&goals=0&target_all=yes : &goals=0&target_all=yes : : -: : : : : 7 : : -: : : : : 3 : : -: : : : howCampStat&cid=4121349&detail=Yes&types=days : owCampStat&cid=4121349&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-07-08:2013-07-09,2013-06-01:2013-06-06,2013-04-01:2013-06-03&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : : owClients : : -: : : : 15 : : : -: : : : l=Yes&types=days&cid=7103094 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7103094&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 4 : : -: : : : 84 : : : -: : : : 89 : : : -: : : : 1&mediaType=text : : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 2 : : -: : : : 86 : : : -: : : : : 4 : : -: : : : 12 : : : -: : : : : 6 : : -: : : : : owCamps&ulogin=dc12a : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 9 : : -: : : : l=Yes&types=days&cid=141133 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=141133&sho : : -: : : : : w_banners_stat=1 : : -: : : : : 108927&cmd=showCamp : : -: : : : howCamp&cid=6886521&tab=off : owCamp&cid=6886521 : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=7133822 : : -: : : : : owCamps&ulogin=mitsubishi-galant2013 : : -: : : : ditCamp&cid=7088102 : owCamp&cid=7088102 : : -: : : : l=Yes&types=days&cid=5033617 : : : -: : : : : 2 : : -: : : : 56 : : : -: : : : : =Yes&types=days&cid=5056684 : : -: : : : 25 : : : -: : : : : owCamps : : -: : : : l=Yes&types=days&cid=6479409 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=6479409&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 9 : : -: : : : : 85.0 : : -: : : : : =Yes&types=days&cid=7133333 : : -: : : : : owCampStat&cid=3618668&detail=Yes&types=days : : -: : : : 74 : owCamp&cid=4275274&tag=303211 : : -: : : : : =Yes&types=days&cid=3298907 : : -: : : : : 9 : : -: : : : : 1 : : -: : : : howCamp&cid=5931713&tab=wait : owCamp&cid=5931713&tab=active : : -: : : : : &mediaType=text : : -: : : : 22 : : : -: : : : 002.0 : 7 : : -: : : : : e_stat=0&online_stat=0&stat_type=campdate&group=day&target_0=1&targ : : -: : : : : et_1=1 : : -: : : : : 4 : : -: : : : mportCampSuccess&cid=4202002&make_new_camp=0 : owCamp&cid=4202002 : : -: : : : : 5 : : -: : : : 87 : : : -: : : : howCamps : owCamps : : -: : : : 99 : : : -: : : : : 3 : : -: : : : 90 : 0 : : -: : : : edirlevel=1373611946.0&authredirlevel=1372140313.0&authredirlevel= : : : -: : : : 1371447906.0&authredirlevel=1368426548.0&authredirlevel=1367818443 : : : -: : : : .0&cmd=showCamps&ulogin=Newdesign-Larus : : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : 97 : : : -: : : : : 9 : : -: : : : howCamps&ulogin=ogurcova-mkt : =Yes&types=days&cid=4139609&ulogin=ogurcova-mkt : : -: : : : : 2 : : -: : : : : 2 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : : 136655&cmd=showCamp : : -: : : : : 8 : : -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -1000 tuples (7.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageVi ews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -: icso : chen : ngin : : : iews : -: urce : gine : eid : : : : -: id : id : : : : : -+======+======+======+====================================================================+=====================================================================+=======+ -: : : : : ype=campdate : : -: : : : type=campdate : : : -: : : : : &payment_type=1 : : -: : : : : d=7 : : -: : : : : &mediaType=text : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0 : : : -: : : : : 79.0&authredirlevel=1373600157.0&authredirlevel=1373081665.0&authre : : -: : : : : dirlevel=1372822331.0&authredirlevel=1372563095.0&authredirlevel=13 : : -: : : : : 72303801.0&authredirlevel=1372044532.0&authredirlevel=1371730939.0& : : -: : : : : authredirlevel=1371471570.0&authredirlevel=1371212358.0&cmd=showCam : : -: : : : : p&cid=2856748 : : -: : : : t : : : -: : : : : ype=campdate : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : -: : : : : s=1 : : -: : : : : owCamps : : -: : : : 1&mediaType=text : : : -: : : : : dirlevel=1373731047.0&cmd=showCamps : : -: : : : : 09.0&authredirlevel=1370955451.0&cmd=showCamp&cid=5477450 : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : type=campdate : : : -: : : : : d=7 : : -: : : : : 47.0&cmd=showCamp&cid=4567586 : : -: : : : : 6 : : -: : : : : OqWshMF.pl?cmd=showCamps&emedia_camps=0&ulogin=lena-san-std : : -: : : : : n= : : -: : : : : 82.0&cmd=showCamp&cid=2753896 : : -: : : : : 5 : : -: : : : 75 : : : -: : : : : &mediaType=text : : -: : : : : 0 : : -: : : : : 2 : : -: : : : 19 : : : -: : : : 93 : : : -: : : : : 9 : : -: : : : : dirlevel=1373869482.0&cmd=showCamp&cid=2997085 : : -: : : : : 3 : : -: : : : 21 : : : -: : : : : 88.0 : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 01.0&cmd=showCamp&cid=4567586 : : -: : : : howCampStat&cid=7056273&offline_stat=0&online_stat=1&stat_periods= : 56273&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y2= : : -: : : : 2013-05-25:2013-06-24,2013-05-22:2013-05-23,2013-05-17:2013-05-18& : 13&group=day : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : : -: : : : &goals=0&target_all=yes : : : -: : : : ks=1 : : : -: : : : : 1 : : -: : : : : 1 : : -: : : : : 7 : : -: : : : =total&target_all=1&page=1&cmd=showCampStat&cid=3016175&show_banne : owCampStat&cid=3016175&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : rs_stat=1 : 12-02-03:2012-02-04,2011-08-01:2011-09-04,2010-12-29:2011-01-02&typ : : -: : : : : es=total&page=&group=day&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&sh : : -: : : : : ow_banners_stat=1 : : -: : : : 37 : : : -: : : : 65 : : : -: : : : 01 : : : -: : : : : 5 : : -: : : : : 9 : : -: : : : 19 : : : -: : : : 71 : : : -: : : : : 1 : : -: : : : 08 : : : -: : : : 81 : : : -: : : : : 1 : : -: : : : 45 : : : -: : : : : 0 : : -: : : : : 5 : : -: : : : : 8 : : -: : : : type=campdate : : : -: : : : 90 : gn_direct_plastik_den : : -: : : : : 3 : : -: : : : 40 : : : -: : : : 39 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 03 : : : -: : : : : 6 : : -: : : : ks=1 : : : -: : : : : 7 : : -: : : : : 2 : : -: : : : howCamp&cid=5940082 : : : -: : : : : 9 : : -: : : : 7-.31620-2.1373861604902917.198343986.198343986..64036780.1406.9b1 : 7 : : -: : : : : 1 : : -: : : : : Informers.html : : -: : : : d : : : -: : : : 29 : : : -: : : : 95 : : : -: : : : ransfer : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6870158 : : -: : : : 68 : : : -: : : : : dirlevel=1342185655.0&cmd=showCamps&ulogin=beautyland-artha8 : : -: : : : : 1 : : -: : : : : 8 : : -: : : : 71 : : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 4 : : -: : : : : 018058&cmd=showCamp : : -: : : : 76 : owCampStat&cid=5109476&detail=Yes&types=days : : -: : : : : 9 : : -: : : : : owSubClientCamps : : -: : : : : 1 : : -: : : : : 0 : : -: : : : type=campdate : : : -: : : : 91 : : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=4718384 : : -: : : : : w.advertising.yandex.ru/welcome/pdf/direct_booklet.pdf : : -: : : : : =Yes&types=days&cid=7114664 : : -: : : : : 7 : : -: : : : : &mediaType=text : : -: : : : : 51.0 : : -: : : : ditBannerEasy&cid=7080496&bid=188854799 : 080496&cmd=showCamp : : -: : : : : 8 : : -: : : : 56 : owCampStat&cid=5895656&detail=Yes&types=days : : -: : : : edirlevel=1373786750.0&cmd=showCampStat&cid=6421801&cid=6424294&ci : : : -: : : : d=6487283&cid=6493829&cid=6494462&cid=6495189&stat_type=campdate&c : : : -: : : : srf_token=CzqM8_OaIwxa28-a : : : -: : : : 28 : : : -: : : : : 6 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : type=campdate : : : -: : : : 77 : lshoj-kvartiry : : -: : : : 83 : : : -: : : : : vancedForecast : : -: : : : howCamp&cid=2925555 : owCamp&cid=2925555 : : -: : : : howCamps&ulogin=aplarru : 9&ulogin=aplarru : : -: : : : : 1 : : -: : : : 16 : : : -: : : : : ger : : -: : : : : 52.0&authredirlevel=1373884890.0 : : -: : : : l=Yes&types=days&cid=7026863 : : : -: : : : : 4 : : -: : : : : 5 : : -: : : : 43 : : : -: : : : : 1 : : -: : : : : 9 : : -: : : : : owCamp&cid=7129447#193577203 : : -: : : : 38 : 46.0 : : -: : : : : 8 : : -: : : : 44 : : : -: : : : : 8 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6486629 : : -: : : : : 65.0 : : -: : : : : =Yes&types=days&cid=7131552 : : -: : : : : 3 : : -: : : : howCamp&cid=7129026&tab=wait : owCamp&cid=7129026&tab=active : : -: : : : : 3 : : -: : : : : =Yes&types=days&cid=7015594 : : -: : : : 48 : : : -: : : : : 5 : : -: : : : : 3 : : -: : : : 78 : : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=5826713 : : : -: : : : : 5 : : -: : : : : 1 : : -: : : : : =Yes&types=days&cid=5309194 : : -: : : : howCamps : owCamps : : -: : : : : 6 : : -: : : : 76 : : : -: : : : 21 : : : -: : : : 55 : : : -: : : : : 5 : : -: : : : : 1 : : -: : : : : 8 : : -: : : : : 6 : : -: : : : : 6 : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : : -: : : : : type=campdate : : -: : : : t : : : -: : : : l=Yes&types=days&cid=6752443 : 52443&phrasedate=Yes&target_all=1 : : -: : : : howCamp&cid=6501536&tab=&page=2 : : : -: : : : : 8 : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=7026863 : : -: : : : 41 : : : -: : : : : 5 : : -: : : : : owCamps&ulogin=super-svictor : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6411871 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6189242 : : -: : : : l=Yes&cid=6009684&showCampStat=cmd&types=days&target_all=1&d1=08&m : owCampStat&cid=6009684&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : 1=07&y1=13&d2=08&m2=07&y2=13&group=day : 13-02-25:2013-02-25,2013-02-26:2013-02-26,&detail=yes&page=&y1=2013 : : -: : : : : &m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : 1&mediaType=text : : : -: : : : : =Yes&types=days&cid=6752443 : : -: : : : 42 : : : -: : : : : 3 : : -: : : : howCamps : 2 : : -: : : : 263.0&cmd=showCamp&cid=5830159 : owCampStat&cid=5830159&detail=Yes&types=days : : -: : : : : 7 : : -: : : : : =Yes&types=days&cid=6467205 : : -: : : : : =Yes&types=days&cid=6467203 : : -: : : : 89 : : : -: : : : howCamp&cid=5088603 : owCamp&cid=5088603 : : -: : : : 77 : : : -: : : : : 6 : : -: : : : : 78.0 : : -: : : : 42 : : : -: : : : 38 : owCampStat&cid=6356238&detail=Yes&types=days : : -: : : : howCamps : 6 : : -: : : : : 9 : : -: : : : howCamps&ulogin=discoverycruise : =Yes&types=days&cid=7065506&ulogin=discoverycruise : : -: : : : : 33.0&cmd=showCampStat&detail=Yes&types=days&cid=6393197 : : -: : : : : 7 : : -: : : : ditCamp&cid=5309194 : =Yes;types=days;cid=5309194 : : -: : : : : owCamp&cid=6980076 : : -: : : : : dirlevel=1373863808.0&authredirlevel=1373522397.0&cmd=showCamp&cid= : : -: : : : : 6051926&tab=active : : -: : : : : =easy&from=metrika : : -: : : : : =Yes&types=days&cid=7135716 : : -: : : : : 3 : : -: : : : howCamp&cid=7129447&tab=wait : owCamp&cid=7129447&tab=active : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : 5279956&tab=media : owCamp&cid=5279956&tab=active : : -: : : : ditBannerEasy&cid=6835814&bid=171781249 : 835814&cmd=showCamp : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6931378 : : -: : : : : ype=campdate : : -: : : : 68 : : : -: : : : 42 : : : -: : : : l=Yes&types=days&cid=4718384 : : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 7 : : -: : : : : owCamps&ulogin=visacity08 : : -: : : : : 109741&cmd=showCamp : : -: : : : type=campdate&&ulogin=vnukovo-piter : owCamps&ulogin=vnukovo-piter : : -: : : : : 1 : : -: : : : 3243479&tab=media : owCamp&cid=3243479&tab=active : : -: : : : : 1 : : -: : : : ditBannerEasy&cid=7123828&bid=193277936 : 123828&cmd=showCamp : : -: : : : : 6 : : -: : : : : 0 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : howCamp&cid=7133264 : owCamp&cid=7133264&tab=wait : : -: : : : : 5 : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : howCamps&ulogin=well-mir-ru : 0&ulogin=well-mir-ru : : -: : : : 66 : : : -: : : : 83 : : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : 5 : : -: : : : : 4 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : : owCamps&ulogin=modul-sdskbeton : : -: : : : : =Yes&types=days&cid=7064245 : : -: : : : &ulogin= : : : -: : : : : 3 : : -: : : : howCampStat&cid=3370690&detail=Yes&types=days : 0 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : : 3 : : -: : : : : owCamps : : -: : : : : 7 : : -: : : : : come=yes : : -: : : : : 3 : : -: : : : : 9 : : -: : : : : 3 : : -: : : : : 6 : : -: : : : howCampStat&cid=7109552&detail=Yes&types=days : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7109552&sh : : -: : : : : ow_banners_stat=1 : : -: : : : howCamp&cid=7134723 : : : -: : : : howCamps&ulogin=alfenjoy-dir : 3&ulogin=alfenjoy-dir : : -: : : : howCamps&ulogin=allstarshop-yd : owCamps&ulogin=allstarshop-yd : : -: : : : howCamps : 7 : : -: : : : : 6 : : -: : : : 35 : : : -: : : : 96&ulogin=kuncevo-ru : vo-avto-obyavlyaet-start-prodazh-LEXUS-ES-i-LEXUS-RX-spetsialnyh-ve : : -: : : : : rsiy.tmex : : -: : : : : =Yes&types=days&cid=6144446 : : -: : : : : 2 : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 137413&cmd=orderCamp&agree=yes : : : -: : : : : 8 : : -: : : : 80 : : : -: : : : 994219&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y : 9 : : -: : : : 2=13&group=day : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=3346173 : : -: : : : : =Yes&types=days&cid=4300226 : : -: : : : 90 : : : -: : : : id=7 : : : -: : : : : owCampStat&cid=4194803&cid=5256554&cid=5718709&cid=6790267&cid=6790 : : -: : : : : 271&cid=7070376&stat_type=campdate&ulogin=mybalcony-ru&csrf_token=L : : -: : : : : BcSJLrXLKh9zXYq : : -: : : : 38 : owCampStat&cid=6356238&detail=Yes&types=days : : -: : : : 431468&phrasedate=Yes&target_all=1 : owCampStat&cid=5431468&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-06-27:2013-07-10,2013-01-01:2013-06-01,2013-06-01:2013-07-03&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 3 : : -: : : : 74 : : : -: : : : all : : : -: : : : : 7 : : -: : : : : 0 : : -: : : : 16 : : : -: : : : : owClients : : -: : : : : 6 : : -: : : : : =Yes&types=days&cid=1475303 : : -: : : : : 2 : : -: : : : l=Yes&types=days&cid=6942653 : owCampStat&cid=6942653&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-06-22:2013-06-22,2013-06-16:2013-06-18,2013-06-17:2013-06-18&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : 97 : : : -: : : : howCamps : 6 : : -: : : : 51 : : : -: : : : : 480202&cmd=showCamp : : -: : : : howCamp&cid=3973331&ulogin=kuncevo-ru : : : -: : : : : owCamps&ulogin=visamaster08 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=6800439 : : : -: : : : 09 : owCamp&cid=6797009 : : -: : : : : 9 : : -: : : : : dirlevel=1373859813.0&cmd=showCampStat&cid=5538008&detail=Yes&types : : -: : : : : =days : : -: : : : 316.0&cmd=showCamp&cid=5993938 : : : -: : : : : 2 : : -: : : : : 7 : : -: : : : : =Yes&types=days&cid=7103125 : : -: : : : : 8 : : -: : : : howCamp&cid=7129447&tab=wait : owCamp&cid=7129447&tab=active : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6883763 : : -: : : : : 7 : : -: : : : : ype=campdate : : -: : : : : =Yes&types=days&cid=1724608 : : -: : : : : 5 : : -: : : : : 2 : : -: : : : : 3 : : -: : : : : 2 : : -: : : : 09 : : : -: : : : : 4 : : -: : : : : ed : : -: : : : = : 17.0&authredirlevel=1373534053.0&cmd=apiApplicationList&ulogin= : : -: : : : : 2 : : -: : : : HB.31620-2.1373870178024287.199046816.199046816..14150262.1406.c65 : : : -: : : : &rnd=0.9338765616586568 : : : -: : : : 368.0&cmd=showCamp&cid=4685945 : 74.0&authredirlevel=1363929368.0&cmd=showCamp&cid=4685945 : : -: : : : : 5 : : -: : : : : 97.0&cmd=ForecastByWords : : -: : : : 71 : : : -: : : : howCampStat&cid=2196876&detail=Yes&types=days : 6 : : -: : : : = : : : -: : : : howCampStat&cid=7056273&offline_stat=0&online_stat=1&stat_periods= : 56273&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y2= : : -: : : : 2013-05-25:2013-06-24,2013-05-22:2013-05-23,2013-05-17:2013-05-18& : 13&group=day : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : : -: : : : &goals=0&target_all=yes : : : -: : : : : 9 : : -: : : : edirlevel=1373868161.0&&cid=7027214&cmd=showCamp : owCampStat&cid=7027214&detail=Yes&types=days : : -: : : : : =Yes&types=days&cid=6630335 : : -: : : : : owCamps : : -: : : : nt_country=225&cmd=editBannerEasy¤cy=YND_FIXED&interface_typ : 136912&cmd=showCamp&unglued=194242390 : : -: : : : e=easy¬new=yes : : : -: : : : : 7 : : -: : : : : 6 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-11:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-03:201 : : -: : : : 013-07-15,2013-07-01:2013-07-14,2013-07-05:2013-07-13&stat_type=ca : 3-07-15,2013-07-11:2013-07-15,2013-07-01:2013-07-14&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&target_al : ate&y1=2013&m1=07&d1=03&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : l=1 : : : -: : : : : 83.0 : : -: : : : howCamps&ulogin=nordex-grandit : 57.0&cmd=showCampStat&detail=Yes&types=days&cid=6947313&ulogin=nord : : -: : : : : ex-grandit : : -: : : : : =Yes&types=days&cid=7068093 : : -: : : : : =Yes&types=days&cid=6834748 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2011-10-27:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2011-10-27:201 : : -: : : : 011-10-27,2011-08-31:2011-08-31,2011-08-26:2011-08-26&stat_type=ca : 1-10-27,2011-08-31:2011-08-31,2011-08-26:2011-08-26&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=14&y2=2013&m2=07&d2=14&group=day&target_al : ate&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : l=1 : : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 38 : 07.0&authredirlevel=1373876446.0 : : -: : : : : 134782&cmd=showCamp&fixated=150055634 : : -: : : : : dirlevel=1364366494.0&cmd=showCamp&cid=6106774 : : -: : : : : =Yes&types=days&cid=7072622 : : -: : : : : owClients : : -: : : : : 113103&cmd=showCamp : : -: : : : : 2 : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : : 5 : : -: : : : : 05.6&authredirlevel=1372014905.5&authredirlevel=1372014905.4&authre : : -: : : : : dirlevel=1372014905.3&authredirlevel=1372014905.2&authredirlevel=13 : : -: : : : : 72014905.1&authredirlevel=1372014905.0&authredirlevel=1370480737.6& : : -: : : : : authredirlevel=1370480737.5&authredirlevel=1370480737.4&authredirle : : -: : : : : vel=1370480737.3&authredirlevel=1370480737.2&authredirlevel=1370480 : : -: : : : : 737.1&authredirlevel=1370480737.0&cmd=showCampStat&stat_type=campda : : -: : : : : te&target_0=1&target_1=1&target_all=1 : : -: : : : : ulogin= : : -: : : : : 5 : : -: : : : howCamp&cid=6980076 : owCampStat&cid=6980076&detail=Yes&types=days : : -: : : : &ulogin= : ulogin= : : -: : : : howClients : owCamps&ulogin=K-PV-beeline : : -: : : : l=Yes&types=days&cid=88689 : : : -: : : : 32 : : : -: : : : 15 : owCampStat&cid=5991615&detail=Yes&types=days : : -: : : : howCamp&cid=6980076 : owCamp&cid=6980076 : : -: : : : : =Yes&types=days&cid=5871623 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 8 : : -: : : : : owCampStat&cid=4917291&detail=Yes&types=days : : -: : : : : 2 : : -: : : : : 90.0 : : -: : : : : 8 : : -: : : : : owCamps&ulogin=domusmebel-ru : : -: : : : : 0 : : -: : : : : =Yes&types=days&cid=4735372 : : -: : : : : 9 : : -: : : : 49 : : : -: : : : : 5 : : -: : : : 08 : : : -: : : : howCampStat&cid=3353427&offline_stat=0&online_stat=0&stat_periods= : d2=15&target_0=0&stat_periods=2013-07-01:2013-07-06,2013-06-27:2013 : : -: : : : 2013-07-01:2013-07-06,2013-06-27:2013-07-05,2013-06-01:2013-06-30& : -07-05,2013-06-01:2013-06-30&m2=07&stat_type=custom&onpage=100&grou : : -: : : : stat_type=custom&sort=&reverse=ARRAY(0x1639e200)&group_by=tag&filt : p_by=tag, banner, page, phrase&d1=15&target_1=0&cmd=showCampStat&ta : : -: : : : er_tag=&group_by=banner&filter_banner=&group_by=page&filter_page=& : rget_all=1&online_stat=0&m1=07&cid=3353427&goals=0&offline_stat=0&y : : -: : : : filter_page_target=&group_by=phrase&filter_phrase=&goals=0&y1=2013 : 2=13&sort=clicks&reverse=1 : : -: : : : &m1=07&d1=15&y2=2013&m2=07&d2=15&onpage=100 : : : -: : : : : owCamps : : -: : : : : 2 : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 8 : : -: : : : 44 : owCampStat&cid=4154744&detail=Yes&types=days : : -: : : : 01 : : : -: : : : site_promotion_search : : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 02.0 : : -: : : : : 7 : : -: : : : howCamp&cid=6522990&tab=off : owCamp&cid=6522990 : : -: : : : : 9 : : -: : : : : 8 : : -: : : : : 9 : : -: : : : : 5 : : -: : : : : 4 : : -: : : : 813.0&cmd=showCamp&cid=5210661 : : : -: : : : : 63.0&cmd=showCamp&cid=5830159 : : -: : : : : =Yes&types=days&cid=6993362 : : -: : : : : vancedForecast : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : 7090667&cmd=showCamp : owCampStat&cid=7090667&detail=Yes&types=days : : -: : : : 77&ulogin=svk-direct : : : -: : : : : 3 : : -: : : : : 1 : : -: : : : howCampStat&cid=5975585&detail=Yes&types=days : 5 : : -: : : : : 8 : : -: : : : : 1 : : -: : : : : 41.0&authredirlevel=1373879619.0&cmd=showCamp&cid=5356346 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-03-01:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-03-01:201 : : -: : : : 013-07-15,2013-06-01:2013-06-14,2013-06-01:2013-07-15&stat_type=ca : 3-07-15,2013-06-01:2013-06-14,2013-06-01:2013-07-15&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=month&target_ : ate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : all=1 : : : -: : : : : owCamps&ulogin=infoprom51 : : -: : : : : 64.0&cmd=showCamp&cid=5028648 : : -: : : : 64 : : : -: : : : : 6 : : -: : : : : 34.0&cmd=ForecastByWords : : -: : : : : 4 : : -: : : : ts : : : -: : : : : 2 : : -: : : : ncy : : : -: : : : : 8 : : -: : : : : 55.0&cmd=showCampStat&detail=Yes&types=days&cid=6849113 : : -: : : : howCamps&ulogin=aplarru : 9&ulogin=aplarru : : -: : : : : =Yes&types=days&cid=5039376 : : -: : : : : owCamp&cid=4748947&ulogin=cont5nn#71488518 : : -: : : : _camps=0&cmd=showSubClientCamps&authredirlevel=1373873308.0&settin : dirlevel=1373880608.0&media_camps=0&cmd=showSubClientCamps&authredi : : -: : : : gs_sorted=&sort=total&reverse=1 : rlevel=1373873308.0&settings_sorted=&sort=total&reverse=0 : : -: : : : howCamp&cid=4581573&tab=decline : owCamp&cid=4581573&tab=active : : -: : : : : 9 : : -: : : : howClients : vancedForecast : : -: : : : : 983763&cmd=showCamp&unglued=194305392 : : -: : : : : 9 : : -: : : : : 5 : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=7030009 : : -: : : : 0 : : : -: : : : : 8 : : -: : : : : 9 : : -: : : : : 65.0&cmd=showCamp&cid=7135509 : : -: : : : 642.0 : ype=campdate : : -: : : : : 5 : : -: : : : 35 : 5 : : -: : : : : dirlevel=1373877895.0&cmd=showCamp&cid=3055906 : : -: : : : : 6 : : -: : : : 099442&cid=7100140&cid=7100144&cid=7100148&cid=7100150&cid=7132172 : owCampStat&cid=7099442&cid=7100140&cid=7100144&cid=7100148&cid=7100 : : -: : : : &cid=7132174&cid=7132175&cid=7132177&cid=7132179&cid=7132632&cid=7 : 150&cid=7132172&cid=7132174&cid=7132175&cid=7132177&cid=7132179&cid : : -: : : : 132838&cid=7132888&cid=7133059&cid=7133217&stat_type=campdate&csrf : =7132632&cid=7132838&cid=7132888&cid=7133059&cid=7133217&offline_st : : -: : : : _token=aENmaxzslTI5oMqD : at=0&online_stat=0&stat_periods=2013-07-01:2013-07-13,2013-07-01:20 : : -: : : : : 13-07-12,2013-06-14:2013-07-11&stat_type=campdate&y1=2013&m1=07&d1= : : -: : : : : 15&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : 35 : 63.0&authredirlevel=1373873632.0 : : -: : : : : 1 : : -: : : : 73 : : : -: : : : howCampStat&cid=6981382&offline_stat=0&online_stat=1&stat_periods= : owCampStat&cid=6981382&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : 2013-06-10:2013-06-25,2013-06-10:2013-06-24,&detail=yes&page=&y1=2 : 13-06-10:2013-06-25,2013-06-10:2013-06-24,&detail=yes&page=&y1=2013 : : -: : : : 013&m1=07&d1=05&y2=2013&m2=07&d2=12&group=day&goals=0&target_all=y : &m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : es : : : -: : : : : =Yes&types=days&cid=6621650 : : -: : : : gSearch&who=camps&cid=&camp_login=&camp_name=атлант : 4&ulogin=atlant-b2bnew : : -: : : : : 0 : : -: : : : : owCamps&ulogin=domusmebel-ru : : -: : : : : 3 : : -: : : : : owCamp&cid=3601040 : : -: : : : : =Yes&types=days&cid=6858643 : : -: : : : : =Yes&types=days&cid=7109462 : : -: : : : : =Yes&types=days&cid=7085888 : : -: : : : 25 : : : -: : : : : owCamps : : -: : : : d=18770681&resp_desc=success : owCamps : : -: : : : howCamp&cid=6980076 : owCampStat&cid=6980076&detail=Yes&types=days : : -: : : : : 5 : : -: : : : : 4 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=5129796 : : -: : : : : 7 : : -: : : : howCampStat&cid=7104084&detail=Yes&types=days : 104084&cmd=showCamp : : -: : : : : 6 : : -: : : : : 2 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=3427083 : : : -: : : : : =Yes&types=days&cid=7060114 : : -: : : : l=Yes&types=days&cid=6833696 : owCampStat&cid=6833696&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-20:2013-06-21,&detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m : : -: : : : : 2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : : 137972&cmd=showCamp&unglued=194336987 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : &mediaType=text&firstbanner=std : : -: : : : 137830&cmd=orderCamp&agree=yes : : : -: : : : : 9 : : -: : : : 09 : : : -: : : : : 131075&cmd=showCamp : : -: : : : : 2 : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6960697 : : -: : : : 7129022&cmd=showCamp : owCampStat&cid=7129022&detail=Yes&types=days : : -: : : : 43 : : : -: : : : : 8 : : -: : : : 17 : owCamp&cid=7082517 : : -: : : : : 6 : : -: : : : 31&ulogin=kuncevo-ru : : : -: : : : 394451&phrasedate=Yes&target_all=1 : owCampStat&cid=4394451&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 12-01-01:2012-11-21,2012-01-01:2012-07-31,2012-03-23:2012-03-23&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 1 : : -: : : : howCamps : owCamps : : -: : : : : 1 : : -: : : : : 6 : : -: : : : gSearch&who=camps&cid=&camp_login=&camp_name=дизв : 9&ulogin=dizvitr-b2b : : -: : : : : 7 : : -: : : : : 9 : : -: : : : : campdate : : -: : : : : 0 : : -: : : : : 6 : : -: : : : : 6 : : -: : : : howCamps&ulogin=oknabuduschego : =Yes&types=days&cid=5506368&ulogin=oknabuduschego : : -: : : : howClients&sort=total&reverse=0 : owCamps&ulogin=Rusokon2007 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=4879898 : : -: : : : 1&mediaType=text : : : -: : : : 85 : : : -: : : : howCamps&ulogin=visacity08 : 5&ulogin=visacity08 : : -: : : : 56 : : : -: : : : : owCamps&ulogin=dc12a : : -: : : : 59 : : : -: : : : howCamps&ulogin=ul-smilestar-su : =ul-smilestar-su#list : : -: : : : howCampStat&cid=7053221&detail=Yes&types=days : owCampStat&cid=7053221&offline_stat=0&online_stat=0&stat_periods=&d : : -: : : : : etail=yes&page=&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&g : : -: : : : : oals=0&target_all=yes&show_banners_stat=1 : : -: : : : : =Yes&types=days&cid=7133318 : : -: : : : : 5 : : -: : : : 25 : : : -: : : : howCamp&tab=all&cid=7126549&ulogin=transkontinental2014#194270034 : : : -: : : : : 2 : : -: : : : 02 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 07 : : : -: : : : : 1 : : -: : : : : 9 : : -: : : : howCamp&cid=7078898&tab=off : owCamp&cid=7078898 : : -: : : : 01 : : : -: : : : : 4 : : -: : : : : 35.0 : : -: : : : : owCampStat&cid=2886087&detail=Yes&types=days : : -: : : : : 8 : : -: : : : type=campdate : dirlevel=1373865822.0&cmd=showCampStat&offline_stat=0&online_stat=0 : : -: : : : : &stat_periods=2013-07-04:2013-07-05,2013-06-10:2013-06-13,2013-05-2 : : -: : : : : 1:2013-05-31&stat_type=campdate&y1=2013&m1=07&d1=15&y2=2013&m2=07&d : : -: : : : : 2=15&group=day&target_all=1 : : -: : : : : 0 : : -: : : : : owCamp&cid=6354573#194189392 : : -: : : : : owCamps : : -: : : : 25 : : : -: : : : : 8 : : -: : : : : 04.0&cmd=showCamp&cid=7129428 : : -: : : : howCamp&cid=7130323 : ids=193775525&cid=7130323 : : -: : : : howCamps&ulogin=richton-studio : 7&ulogin=richton-studio : : -: : : : : owCampStat&cid=7034406&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-07-11:2013-07-11,2013-07-12:2013-07-12,2013-07-13:2013-07-13&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 0 : : -: : : : howCamp&tab=decline&cid=6886514 : owCamp&cid=6886514&tab=active : : -: : : : : 33.0 : : -: : : : : 60.0 : : -: : : : : 5 : : -: : : : : dirlevel=1373866480.0&cmd=showCampStat&stat_type=by_agency_clients : : -: : : : : 9 : : -: : : : : 463697&cmd=showCamp : : -: : : : 28 : : : -: : : : : 6 : : -: : : : : 8 : : -: : : : : 4 : : -: : : : 60 : : : -: : : : : 5 : : -: : : : : 7 : : -: : : : : 1 : : -: : : : howCampStat&cid=6456112&offline_stat=0&online_stat=0&stat_periods= : owCampStat&cid=6456112&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : 2013-07-10:2013-07-10,2013-07-09:2013-07-09,2013-07-08:2013-07-08& : 13-07-10:2013-07-10,2013-07-09:2013-07-09,2013-07-08:2013-07-08&phr : : -: : : : phrasedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : =day&goals=0&target_all=yes : &goals=0&target_all=yes : : -: : : : : 7 : : -: : : : : 3 : : -: : : : howCampStat&cid=4121349&detail=Yes&types=days : owCampStat&cid=4121349&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-07-08:2013-07-09,2013-06-01:2013-06-06,2013-04-01:2013-06-03&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : : owClients : : -: : : : 15 : : : -: : : : l=Yes&types=days&cid=7103094 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7103094&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 4 : : -: : : : 84 : : : -: : : : 89 : : : -: : : : 1&mediaType=text : : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 2 : : -: : : : 86 : : : -: : : : : 4 : : -: : : : 12 : : : -: : : : : 6 : : -: : : : : owCamps&ulogin=dc12a : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 9 : : -: : : : l=Yes&types=days&cid=141133 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=141133&sho : : -: : : : : w_banners_stat=1 : : -: : : : : 108927&cmd=showCamp : : -: : : : howCamp&cid=6886521&tab=off : owCamp&cid=6886521 : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=7133822 : : -: : : : : owCamps&ulogin=mitsubishi-galant2013 : : -: : : : ditCamp&cid=7088102 : owCamp&cid=7088102 : : -: : : : l=Yes&types=days&cid=5033617 : : : -: : : : : 2 : : -: : : : 56 : : : -: : : : : =Yes&types=days&cid=5056684 : : -: : : : 25 : : : -: : : : : owCamps : : -: : : : l=Yes&types=days&cid=6479409 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=6479409&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 9 : : -: : : : : 85.0 : : -: : : : : =Yes&types=days&cid=7133333 : : -: : : : : owCampStat&cid=3618668&detail=Yes&types=days : : -: : : : 74 : owCamp&cid=4275274&tag=303211 : : -: : : : : =Yes&types=days&cid=3298907 : : -: : : : : 9 : : -: : : : : 1 : : -: : : : howCamp&cid=5931713&tab=wait : owCamp&cid=5931713&tab=active : : -: : : : : &mediaType=text : : -: : : : 22 : : : -: : : : 002.0 : 7 : : -: : : : : e_stat=0&online_stat=0&stat_type=campdate&group=day&target_0=1&targ : : -: : : : : et_1=1 : : -: : : : : 4 : : -: : : : mportCampSuccess&cid=4202002&make_new_camp=0 : owCamp&cid=4202002 : : -: : : : : 5 : : -: : : : 87 : : : -: : : : howCamps : owCamps : : -: : : : 99 : : : -: : : : : 3 : : -: : : : 90 : 0 : : -: : : : edirlevel=1373611946.0&authredirlevel=1372140313.0&authredirlevel= : : : -: : : : 1371447906.0&authredirlevel=1368426548.0&authredirlevel=1367818443 : : : -: : : : .0&cmd=showCamps&ulogin=Newdesign-Larus : : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : 97 : : : -: : : : : 9 : : -: : : : howCamps&ulogin=ogurcova-mkt : =Yes&types=days&cid=4139609&ulogin=ogurcova-mkt : : -: : : : : 2 : : -: : : : : 2 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : : 136655&cmd=showCamp : : -: : : : : 8 : : -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -1000 tuples (5.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageVi ews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -: icso : chen : ngin : : : iews : -: urce : gine : eid : : : : -: id : id : : : : : -+======+======+======+====================================================================+=====================================================================+=======+ -: : : : : ype=campdate : : -: : : : type=campdate : : : -: : : : : &payment_type=1 : : -: : : : : d=7 : : -: : : : : &mediaType=text : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0 : : : -: : : : : 79.0&authredirlevel=1373600157.0&authredirlevel=1373081665.0&authre : : -: : : : : dirlevel=1372822331.0&authredirlevel=1372563095.0&authredirlevel=13 : : -: : : : : 72303801.0&authredirlevel=1372044532.0&authredirlevel=1371730939.0& : : -: : : : : authredirlevel=1371471570.0&authredirlevel=1371212358.0&cmd=showCam : : -: : : : : p&cid=2856748 : : -: : : : t : : : -: : : : : ype=campdate : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : -: : : : : s=1 : : -: : : : : owCamps : : -: : : : 1&mediaType=text : : : -: : : : : dirlevel=1373731047.0&cmd=showCamps : : -: : : : : 09.0&authredirlevel=1370955451.0&cmd=showCamp&cid=5477450 : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : type=campdate : : : -: : : : : d=7 : : -: : : : : 47.0&cmd=showCamp&cid=4567586 : : -: : : : : 6 : : -: : : : : OqWshMF.pl?cmd=showCamps&emedia_camps=0&ulogin=lena-san-std : : -: : : : : n= : : -: : : : : 82.0&cmd=showCamp&cid=2753896 : : -: : : : : 5 : : -: : : : 75 : : : -: : : : : &mediaType=text : : -: : : : : 0 : : -: : : : : 2 : : -: : : : 19 : : : -: : : : 93 : : : -: : : : : 9 : : -: : : : : dirlevel=1373869482.0&cmd=showCamp&cid=2997085 : : -: : : : : 3 : : -: : : : 21 : : : -: : : : : 88.0 : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 01.0&cmd=showCamp&cid=4567586 : : -: : : : howCampStat&cid=7056273&offline_stat=0&online_stat=1&stat_periods= : 56273&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y2= : : -: : : : 2013-05-25:2013-06-24,2013-05-22:2013-05-23,2013-05-17:2013-05-18& : 13&group=day : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : : -: : : : &goals=0&target_all=yes : : : -: : : : ks=1 : : : -: : : : : 1 : : -: : : : : 1 : : -: : : : : 7 : : -: : : : =total&target_all=1&page=1&cmd=showCampStat&cid=3016175&show_banne : owCampStat&cid=3016175&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : rs_stat=1 : 12-02-03:2012-02-04,2011-08-01:2011-09-04,2010-12-29:2011-01-02&typ : : -: : : : : es=total&page=&group=day&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&sh : : -: : : : : ow_banners_stat=1 : : -: : : : 37 : : : -: : : : 65 : : : -: : : : 01 : : : -: : : : : 5 : : -: : : : : 9 : : -: : : : 19 : : : -: : : : 71 : : : -: : : : : 1 : : -: : : : 08 : : : -: : : : 81 : : : -: : : : : 1 : : -: : : : 45 : : : -: : : : : 0 : : -: : : : : 5 : : -: : : : : 8 : : -: : : : type=campdate : : : -: : : : 90 : gn_direct_plastik_den : : -: : : : : 3 : : -: : : : 40 : : : -: : : : 39 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 03 : : : -: : : : : 6 : : -: : : : ks=1 : : : -: : : : : 7 : : -: : : : : 2 : : -: : : : howCamp&cid=5940082 : : : -: : : : : 9 : : -: : : : 7-.31620-2.1373861604902917.198343986.198343986..64036780.1406.9b1 : 7 : : -: : : : : 1 : : -: : : : : Informers.html : : -: : : : d : : : -: : : : 29 : : : -: : : : 95 : : : -: : : : ransfer : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6870158 : : -: : : : 68 : : : -: : : : : dirlevel=1342185655.0&cmd=showCamps&ulogin=beautyland-artha8 : : -: : : : : 1 : : -: : : : : 8 : : -: : : : 71 : : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 4 : : -: : : : : 018058&cmd=showCamp : : -: : : : 76 : owCampStat&cid=5109476&detail=Yes&types=days : : -: : : : : 9 : : -: : : : : owSubClientCamps : : -: : : : : 1 : : -: : : : : 0 : : -: : : : type=campdate : : : -: : : : 91 : : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=4718384 : : -: : : : : w.advertising.yandex.ru/welcome/pdf/direct_booklet.pdf : : -: : : : : =Yes&types=days&cid=7114664 : : -: : : : : 7 : : -: : : : : &mediaType=text : : -: : : : : 51.0 : : -: : : : ditBannerEasy&cid=7080496&bid=188854799 : 080496&cmd=showCamp : : -: : : : : 8 : : -: : : : 56 : owCampStat&cid=5895656&detail=Yes&types=days : : -: : : : edirlevel=1373786750.0&cmd=showCampStat&cid=6421801&cid=6424294&ci : : : -: : : : d=6487283&cid=6493829&cid=6494462&cid=6495189&stat_type=campdate&c : : : -: : : : srf_token=CzqM8_OaIwxa28-a : : : -: : : : 28 : : : -: : : : : 6 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : type=campdate : : : -: : : : 77 : lshoj-kvartiry : : -: : : : 83 : : : -: : : : : vancedForecast : : -: : : : howCamp&cid=2925555 : owCamp&cid=2925555 : : -: : : : howCamps&ulogin=aplarru : 9&ulogin=aplarru : : -: : : : : 1 : : -: : : : 16 : : : -: : : : : ger : : -: : : : : 52.0&authredirlevel=1373884890.0 : : -: : : : l=Yes&types=days&cid=7026863 : : : -: : : : : 4 : : -: : : : : 5 : : -: : : : 43 : : : -: : : : : 1 : : -: : : : : 9 : : -: : : : : owCamp&cid=7129447#193577203 : : -: : : : 38 : 46.0 : : -: : : : : 8 : : -: : : : 44 : : : -: : : : : 8 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6486629 : : -: : : : : 65.0 : : -: : : : : =Yes&types=days&cid=7131552 : : -: : : : : 3 : : -: : : : howCamp&cid=7129026&tab=wait : owCamp&cid=7129026&tab=active : : -: : : : : 3 : : -: : : : : =Yes&types=days&cid=7015594 : : -: : : : 48 : : : -: : : : : 5 : : -: : : : : 3 : : -: : : : 78 : : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=5826713 : : : -: : : : : 5 : : -: : : : : 1 : : -: : : : : =Yes&types=days&cid=5309194 : : -: : : : howCamps : owCamps : : -: : : : : 6 : : -: : : : 76 : : : -: : : : 21 : : : -: : : : 55 : : : -: : : : : 5 : : -: : : : : 1 : : -: : : : : 8 : : -: : : : : 6 : : -: : : : : 6 : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : : -: : : : : type=campdate : : -: : : : t : : : -: : : : l=Yes&types=days&cid=6752443 : 52443&phrasedate=Yes&target_all=1 : : -: : : : howCamp&cid=6501536&tab=&page=2 : : : -: : : : : 8 : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=7026863 : : -: : : : 41 : : : -: : : : : 5 : : -: : : : : owCamps&ulogin=super-svictor : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6411871 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6189242 : : -: : : : l=Yes&cid=6009684&showCampStat=cmd&types=days&target_all=1&d1=08&m : owCampStat&cid=6009684&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : 1=07&y1=13&d2=08&m2=07&y2=13&group=day : 13-02-25:2013-02-25,2013-02-26:2013-02-26,&detail=yes&page=&y1=2013 : : -: : : : : &m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : 1&mediaType=text : : : -: : : : : =Yes&types=days&cid=6752443 : : -: : : : 42 : : : -: : : : : 3 : : -: : : : howCamps : 2 : : -: : : : 263.0&cmd=showCamp&cid=5830159 : owCampStat&cid=5830159&detail=Yes&types=days : : -: : : : : 7 : : -: : : : : =Yes&types=days&cid=6467205 : : -: : : : : =Yes&types=days&cid=6467203 : : -: : : : 89 : : : -: : : : howCamp&cid=5088603 : owCamp&cid=5088603 : : -: : : : 77 : : : -: : : : : 6 : : -: : : : : 78.0 : : -: : : : 42 : : : -: : : : 38 : owCampStat&cid=6356238&detail=Yes&types=days : : -: : : : howCamps : 6 : : -: : : : : 9 : : -: : : : howCamps&ulogin=discoverycruise : =Yes&types=days&cid=7065506&ulogin=discoverycruise : : -: : : : : 33.0&cmd=showCampStat&detail=Yes&types=days&cid=6393197 : : -: : : : : 7 : : -: : : : ditCamp&cid=5309194 : =Yes;types=days;cid=5309194 : : -: : : : : owCamp&cid=6980076 : : -: : : : : dirlevel=1373863808.0&authredirlevel=1373522397.0&cmd=showCamp&cid= : : -: : : : : 6051926&tab=active : : -: : : : : =easy&from=metrika : : -: : : : : =Yes&types=days&cid=7135716 : : -: : : : : 3 : : -: : : : howCamp&cid=7129447&tab=wait : owCamp&cid=7129447&tab=active : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : 5279956&tab=media : owCamp&cid=5279956&tab=active : : -: : : : ditBannerEasy&cid=6835814&bid=171781249 : 835814&cmd=showCamp : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6931378 : : -: : : : : ype=campdate : : -: : : : 68 : : : -: : : : 42 : : : -: : : : l=Yes&types=days&cid=4718384 : : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 7 : : -: : : : : owCamps&ulogin=visacity08 : : -: : : : : 109741&cmd=showCamp : : -: : : : type=campdate&&ulogin=vnukovo-piter : owCamps&ulogin=vnukovo-piter : : -: : : : : 1 : : -: : : : 3243479&tab=media : owCamp&cid=3243479&tab=active : : -: : : : : 1 : : -: : : : ditBannerEasy&cid=7123828&bid=193277936 : 123828&cmd=showCamp : : -: : : : : 6 : : -: : : : : 0 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : howCamp&cid=7133264 : owCamp&cid=7133264&tab=wait : : -: : : : : 5 : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : howCamps&ulogin=well-mir-ru : 0&ulogin=well-mir-ru : : -: : : : 66 : : : -: : : : 83 : : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : 5 : : -: : : : : 4 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : : owCamps&ulogin=modul-sdskbeton : : -: : : : : =Yes&types=days&cid=7064245 : : -: : : : &ulogin= : : : -: : : : : 3 : : -: : : : howCampStat&cid=3370690&detail=Yes&types=days : 0 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : : 3 : : -: : : : : owCamps : : -: : : : : 7 : : -: : : : : come=yes : : -: : : : : 3 : : -: : : : : 9 : : -: : : : : 3 : : -: : : : : 6 : : -: : : : howCampStat&cid=7109552&detail=Yes&types=days : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7109552&sh : : -: : : : : ow_banners_stat=1 : : -: : : : howCamp&cid=7134723 : : : -: : : : howCamps&ulogin=alfenjoy-dir : 3&ulogin=alfenjoy-dir : : -: : : : howCamps&ulogin=allstarshop-yd : owCamps&ulogin=allstarshop-yd : : -: : : : howCamps : 7 : : -: : : : : 6 : : -: : : : 35 : : : -: : : : 96&ulogin=kuncevo-ru : vo-avto-obyavlyaet-start-prodazh-LEXUS-ES-i-LEXUS-RX-spetsialnyh-ve : : -: : : : : rsiy.tmex : : -: : : : : =Yes&types=days&cid=6144446 : : -: : : : : 2 : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 137413&cmd=orderCamp&agree=yes : : : -: : : : : 8 : : -: : : : 80 : : : -: : : : 994219&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y : 9 : : -: : : : 2=13&group=day : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=3346173 : : -: : : : : =Yes&types=days&cid=4300226 : : -: : : : 90 : : : -: : : : id=7 : : : -: : : : : owCampStat&cid=4194803&cid=5256554&cid=5718709&cid=6790267&cid=6790 : : -: : : : : 271&cid=7070376&stat_type=campdate&ulogin=mybalcony-ru&csrf_token=L : : -: : : : : BcSJLrXLKh9zXYq : : -: : : : 38 : owCampStat&cid=6356238&detail=Yes&types=days : : -: : : : 431468&phrasedate=Yes&target_all=1 : owCampStat&cid=5431468&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-06-27:2013-07-10,2013-01-01:2013-06-01,2013-06-01:2013-07-03&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 3 : : -: : : : 74 : : : -: : : : all : : : -: : : : : 7 : : -: : : : : 0 : : -: : : : 16 : : : -: : : : : owClients : : -: : : : : 6 : : -: : : : : =Yes&types=days&cid=1475303 : : -: : : : : 2 : : -: : : : l=Yes&types=days&cid=6942653 : owCampStat&cid=6942653&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-06-22:2013-06-22,2013-06-16:2013-06-18,2013-06-17:2013-06-18&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : 97 : : : -: : : : howCamps : 6 : : -: : : : 51 : : : -: : : : : 480202&cmd=showCamp : : -: : : : howCamp&cid=3973331&ulogin=kuncevo-ru : : : -: : : : : owCamps&ulogin=visamaster08 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=6800439 : : : -: : : : 09 : owCamp&cid=6797009 : : -: : : : : 9 : : -: : : : : dirlevel=1373859813.0&cmd=showCampStat&cid=5538008&detail=Yes&types : : -: : : : : =days : : -: : : : 316.0&cmd=showCamp&cid=5993938 : : : -: : : : : 2 : : -: : : : : 7 : : -: : : : : =Yes&types=days&cid=7103125 : : -: : : : : 8 : : -: : : : howCamp&cid=7129447&tab=wait : owCamp&cid=7129447&tab=active : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6883763 : : -: : : : : 7 : : -: : : : : ype=campdate : : -: : : : : =Yes&types=days&cid=1724608 : : -: : : : : 5 : : -: : : : : 2 : : -: : : : : 3 : : -: : : : : 2 : : -: : : : 09 : : : -: : : : : 4 : : -: : : : : ed : : -: : : : = : 17.0&authredirlevel=1373534053.0&cmd=apiApplicationList&ulogin= : : -: : : : : 2 : : -: : : : HB.31620-2.1373870178024287.199046816.199046816..14150262.1406.c65 : : : -: : : : &rnd=0.9338765616586568 : : : -: : : : 368.0&cmd=showCamp&cid=4685945 : 74.0&authredirlevel=1363929368.0&cmd=showCamp&cid=4685945 : : -: : : : : 5 : : -: : : : : 97.0&cmd=ForecastByWords : : -: : : : 71 : : : -: : : : howCampStat&cid=2196876&detail=Yes&types=days : 6 : : -: : : : = : : : -: : : : howCampStat&cid=7056273&offline_stat=0&online_stat=1&stat_periods= : 56273&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y2= : : -: : : : 2013-05-25:2013-06-24,2013-05-22:2013-05-23,2013-05-17:2013-05-18& : 13&group=day : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : : -: : : : &goals=0&target_all=yes : : : -: : : : : 9 : : -: : : : edirlevel=1373868161.0&&cid=7027214&cmd=showCamp : owCampStat&cid=7027214&detail=Yes&types=days : : -: : : : : =Yes&types=days&cid=6630335 : : -: : : : : owCamps : : -: : : : nt_country=225&cmd=editBannerEasy¤cy=YND_FIXED&interface_typ : 136912&cmd=showCamp&unglued=194242390 : : -: : : : e=easy¬new=yes : : : -: : : : : 7 : : -: : : : : 6 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-11:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-03:201 : : -: : : : 013-07-15,2013-07-01:2013-07-14,2013-07-05:2013-07-13&stat_type=ca : 3-07-15,2013-07-11:2013-07-15,2013-07-01:2013-07-14&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&target_al : ate&y1=2013&m1=07&d1=03&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : l=1 : : : -: : : : : 83.0 : : -: : : : howCamps&ulogin=nordex-grandit : 57.0&cmd=showCampStat&detail=Yes&types=days&cid=6947313&ulogin=nord : : -: : : : : ex-grandit : : -: : : : : =Yes&types=days&cid=7068093 : : -: : : : : =Yes&types=days&cid=6834748 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2011-10-27:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2011-10-27:201 : : -: : : : 011-10-27,2011-08-31:2011-08-31,2011-08-26:2011-08-26&stat_type=ca : 1-10-27,2011-08-31:2011-08-31,2011-08-26:2011-08-26&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=14&y2=2013&m2=07&d2=14&group=day&target_al : ate&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : l=1 : : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 38 : 07.0&authredirlevel=1373876446.0 : : -: : : : : 134782&cmd=showCamp&fixated=150055634 : : -: : : : : dirlevel=1364366494.0&cmd=showCamp&cid=6106774 : : -: : : : : =Yes&types=days&cid=7072622 : : -: : : : : owClients : : -: : : : : 113103&cmd=showCamp : : -: : : : : 2 : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : : 5 : : -: : : : : 05.6&authredirlevel=1372014905.5&authredirlevel=1372014905.4&authre : : -: : : : : dirlevel=1372014905.3&authredirlevel=1372014905.2&authredirlevel=13 : : -: : : : : 72014905.1&authredirlevel=1372014905.0&authredirlevel=1370480737.6& : : -: : : : : authredirlevel=1370480737.5&authredirlevel=1370480737.4&authredirle : : -: : : : : vel=1370480737.3&authredirlevel=1370480737.2&authredirlevel=1370480 : : -: : : : : 737.1&authredirlevel=1370480737.0&cmd=showCampStat&stat_type=campda : : -: : : : : te&target_0=1&target_1=1&target_all=1 : : -: : : : : ulogin= : : -: : : : : 5 : : -: : : : howCamp&cid=6980076 : owCampStat&cid=6980076&detail=Yes&types=days : : -: : : : &ulogin= : ulogin= : : -: : : : howClients : owCamps&ulogin=K-PV-beeline : : -: : : : l=Yes&types=days&cid=88689 : : : -: : : : 32 : : : -: : : : 15 : owCampStat&cid=5991615&detail=Yes&types=days : : -: : : : howCamp&cid=6980076 : owCamp&cid=6980076 : : -: : : : : =Yes&types=days&cid=5871623 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 8 : : -: : : : : owCampStat&cid=4917291&detail=Yes&types=days : : -: : : : : 2 : : -: : : : : 90.0 : : -: : : : : 8 : : -: : : : : owCamps&ulogin=domusmebel-ru : : -: : : : : 0 : : -: : : : : =Yes&types=days&cid=4735372 : : -: : : : : 9 : : -: : : : 49 : : : -: : : : : 5 : : -: : : : 08 : : : -: : : : howCampStat&cid=3353427&offline_stat=0&online_stat=0&stat_periods= : d2=15&target_0=0&stat_periods=2013-07-01:2013-07-06,2013-06-27:2013 : : -: : : : 2013-07-01:2013-07-06,2013-06-27:2013-07-05,2013-06-01:2013-06-30& : -07-05,2013-06-01:2013-06-30&m2=07&stat_type=custom&onpage=100&grou : : -: : : : stat_type=custom&sort=&reverse=ARRAY(0x1639e200)&group_by=tag&filt : p_by=tag, banner, page, phrase&d1=15&target_1=0&cmd=showCampStat&ta : : -: : : : er_tag=&group_by=banner&filter_banner=&group_by=page&filter_page=& : rget_all=1&online_stat=0&m1=07&cid=3353427&goals=0&offline_stat=0&y : : -: : : : filter_page_target=&group_by=phrase&filter_phrase=&goals=0&y1=2013 : 2=13&sort=clicks&reverse=1 : : -: : : : &m1=07&d1=15&y2=2013&m2=07&d2=15&onpage=100 : : : -: : : : : owCamps : : -: : : : : 2 : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 8 : : -: : : : 44 : owCampStat&cid=4154744&detail=Yes&types=days : : -: : : : 01 : : : -: : : : site_promotion_search : : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 02.0 : : -: : : : : 7 : : -: : : : howCamp&cid=6522990&tab=off : owCamp&cid=6522990 : : -: : : : : 9 : : -: : : : : 8 : : -: : : : : 9 : : -: : : : : 5 : : -: : : : : 4 : : -: : : : 813.0&cmd=showCamp&cid=5210661 : : : -: : : : : 63.0&cmd=showCamp&cid=5830159 : : -: : : : : =Yes&types=days&cid=6993362 : : -: : : : : vancedForecast : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : 7090667&cmd=showCamp : owCampStat&cid=7090667&detail=Yes&types=days : : -: : : : 77&ulogin=svk-direct : : : -: : : : : 3 : : -: : : : : 1 : : -: : : : howCampStat&cid=5975585&detail=Yes&types=days : 5 : : -: : : : : 8 : : -: : : : : 1 : : -: : : : : 41.0&authredirlevel=1373879619.0&cmd=showCamp&cid=5356346 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-03-01:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-03-01:201 : : -: : : : 013-07-15,2013-06-01:2013-06-14,2013-06-01:2013-07-15&stat_type=ca : 3-07-15,2013-06-01:2013-06-14,2013-06-01:2013-07-15&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=month&target_ : ate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : all=1 : : : -: : : : : owCamps&ulogin=infoprom51 : : -: : : : : 64.0&cmd=showCamp&cid=5028648 : : -: : : : 64 : : : -: : : : : 6 : : -: : : : : 34.0&cmd=ForecastByWords : : -: : : : : 4 : : -: : : : ts : : : -: : : : : 2 : : -: : : : ncy : : : -: : : : : 8 : : -: : : : : 55.0&cmd=showCampStat&detail=Yes&types=days&cid=6849113 : : -: : : : howCamps&ulogin=aplarru : 9&ulogin=aplarru : : -: : : : : =Yes&types=days&cid=5039376 : : -: : : : : owCamp&cid=4748947&ulogin=cont5nn#71488518 : : -: : : : _camps=0&cmd=showSubClientCamps&authredirlevel=1373873308.0&settin : dirlevel=1373880608.0&media_camps=0&cmd=showSubClientCamps&authredi : : -: : : : gs_sorted=&sort=total&reverse=1 : rlevel=1373873308.0&settings_sorted=&sort=total&reverse=0 : : -: : : : howCamp&cid=4581573&tab=decline : owCamp&cid=4581573&tab=active : : -: : : : : 9 : : -: : : : howClients : vancedForecast : : -: : : : : 983763&cmd=showCamp&unglued=194305392 : : -: : : : : 9 : : -: : : : : 5 : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=7030009 : : -: : : : 0 : : : -: : : : : 8 : : -: : : : : 9 : : -: : : : : 65.0&cmd=showCamp&cid=7135509 : : -: : : : 642.0 : ype=campdate : : -: : : : : 5 : : -: : : : 35 : 5 : : -: : : : : dirlevel=1373877895.0&cmd=showCamp&cid=3055906 : : -: : : : : 6 : : -: : : : 099442&cid=7100140&cid=7100144&cid=7100148&cid=7100150&cid=7132172 : owCampStat&cid=7099442&cid=7100140&cid=7100144&cid=7100148&cid=7100 : : -: : : : &cid=7132174&cid=7132175&cid=7132177&cid=7132179&cid=7132632&cid=7 : 150&cid=7132172&cid=7132174&cid=7132175&cid=7132177&cid=7132179&cid : : -: : : : 132838&cid=7132888&cid=7133059&cid=7133217&stat_type=campdate&csrf : =7132632&cid=7132838&cid=7132888&cid=7133059&cid=7133217&offline_st : : -: : : : _token=aENmaxzslTI5oMqD : at=0&online_stat=0&stat_periods=2013-07-01:2013-07-13,2013-07-01:20 : : -: : : : : 13-07-12,2013-06-14:2013-07-11&stat_type=campdate&y1=2013&m1=07&d1= : : -: : : : : 15&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : 35 : 63.0&authredirlevel=1373873632.0 : : -: : : : : 1 : : -: : : : 73 : : : -: : : : howCampStat&cid=6981382&offline_stat=0&online_stat=1&stat_periods= : owCampStat&cid=6981382&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : 2013-06-10:2013-06-25,2013-06-10:2013-06-24,&detail=yes&page=&y1=2 : 13-06-10:2013-06-25,2013-06-10:2013-06-24,&detail=yes&page=&y1=2013 : : -: : : : 013&m1=07&d1=05&y2=2013&m2=07&d2=12&group=day&goals=0&target_all=y : &m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : es : : : -: : : : : =Yes&types=days&cid=6621650 : : -: : : : gSearch&who=camps&cid=&camp_login=&camp_name=атлант : 4&ulogin=atlant-b2bnew : : -: : : : : 0 : : -: : : : : owCamps&ulogin=domusmebel-ru : : -: : : : : 3 : : -: : : : : owCamp&cid=3601040 : : -: : : : : =Yes&types=days&cid=6858643 : : -: : : : : =Yes&types=days&cid=7109462 : : -: : : : : =Yes&types=days&cid=7085888 : : -: : : : 25 : : : -: : : : : owCamps : : -: : : : d=18770681&resp_desc=success : owCamps : : -: : : : howCamp&cid=6980076 : owCampStat&cid=6980076&detail=Yes&types=days : : -: : : : : 5 : : -: : : : : 4 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=5129796 : : -: : : : : 7 : : -: : : : howCampStat&cid=7104084&detail=Yes&types=days : 104084&cmd=showCamp : : -: : : : : 6 : : -: : : : : 2 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=3427083 : : : -: : : : : =Yes&types=days&cid=7060114 : : -: : : : l=Yes&types=days&cid=6833696 : owCampStat&cid=6833696&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-20:2013-06-21,&detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m : : -: : : : : 2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : : 137972&cmd=showCamp&unglued=194336987 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : &mediaType=text&firstbanner=std : : -: : : : 137830&cmd=orderCamp&agree=yes : : : -: : : : : 9 : : -: : : : 09 : : : -: : : : : 131075&cmd=showCamp : : -: : : : : 2 : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6960697 : : -: : : : 7129022&cmd=showCamp : owCampStat&cid=7129022&detail=Yes&types=days : : -: : : : 43 : : : -: : : : : 8 : : -: : : : 17 : owCamp&cid=7082517 : : -: : : : : 6 : : -: : : : 31&ulogin=kuncevo-ru : : : -: : : : 394451&phrasedate=Yes&target_all=1 : owCampStat&cid=4394451&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 12-01-01:2012-11-21,2012-01-01:2012-07-31,2012-03-23:2012-03-23&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 1 : : -: : : : howCamps : owCamps : : -: : : : : 1 : : -: : : : : 6 : : -: : : : gSearch&who=camps&cid=&camp_login=&camp_name=дизв : 9&ulogin=dizvitr-b2b : : -: : : : : 7 : : -: : : : : 9 : : -: : : : : campdate : : -: : : : : 0 : : -: : : : : 6 : : -: : : : : 6 : : -: : : : howCamps&ulogin=oknabuduschego : =Yes&types=days&cid=5506368&ulogin=oknabuduschego : : -: : : : howClients&sort=total&reverse=0 : owCamps&ulogin=Rusokon2007 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=4879898 : : -: : : : 1&mediaType=text : : : -: : : : 85 : : : -: : : : howCamps&ulogin=visacity08 : 5&ulogin=visacity08 : : -: : : : 56 : : : -: : : : : owCamps&ulogin=dc12a : : -: : : : 59 : : : -: : : : howCamps&ulogin=ul-smilestar-su : =ul-smilestar-su#list : : -: : : : howCampStat&cid=7053221&detail=Yes&types=days : owCampStat&cid=7053221&offline_stat=0&online_stat=0&stat_periods=&d : : -: : : : : etail=yes&page=&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&g : : -: : : : : oals=0&target_all=yes&show_banners_stat=1 : : -: : : : : =Yes&types=days&cid=7133318 : : -: : : : : 5 : : -: : : : 25 : : : -: : : : howCamp&tab=all&cid=7126549&ulogin=transkontinental2014#194270034 : : : -: : : : : 2 : : -: : : : 02 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 07 : : : -: : : : : 1 : : -: : : : : 9 : : -: : : : howCamp&cid=7078898&tab=off : owCamp&cid=7078898 : : -: : : : 01 : : : -: : : : : 4 : : -: : : : : 35.0 : : -: : : : : owCampStat&cid=2886087&detail=Yes&types=days : : -: : : : : 8 : : -: : : : type=campdate : dirlevel=1373865822.0&cmd=showCampStat&offline_stat=0&online_stat=0 : : -: : : : : &stat_periods=2013-07-04:2013-07-05,2013-06-10:2013-06-13,2013-05-2 : : -: : : : : 1:2013-05-31&stat_type=campdate&y1=2013&m1=07&d1=15&y2=2013&m2=07&d : : -: : : : : 2=15&group=day&target_all=1 : : -: : : : : 0 : : -: : : : : owCamp&cid=6354573#194189392 : : -: : : : : owCamps : : -: : : : 25 : : : -: : : : : 8 : : -: : : : : 04.0&cmd=showCamp&cid=7129428 : : -: : : : howCamp&cid=7130323 : ids=193775525&cid=7130323 : : -: : : : howCamps&ulogin=richton-studio : 7&ulogin=richton-studio : : -: : : : : owCampStat&cid=7034406&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-07-11:2013-07-11,2013-07-12:2013-07-12,2013-07-13:2013-07-13&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 0 : : -: : : : howCamp&tab=decline&cid=6886514 : owCamp&cid=6886514&tab=active : : -: : : : : 33.0 : : -: : : : : 60.0 : : -: : : : : 5 : : -: : : : : dirlevel=1373866480.0&cmd=showCampStat&stat_type=by_agency_clients : : -: : : : : 9 : : -: : : : : 463697&cmd=showCamp : : -: : : : 28 : : : -: : : : : 6 : : -: : : : : 8 : : -: : : : : 4 : : -: : : : 60 : : : -: : : : : 5 : : -: : : : : 7 : : -: : : : : 1 : : -: : : : howCampStat&cid=6456112&offline_stat=0&online_stat=0&stat_periods= : owCampStat&cid=6456112&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : 2013-07-10:2013-07-10,2013-07-09:2013-07-09,2013-07-08:2013-07-08& : 13-07-10:2013-07-10,2013-07-09:2013-07-09,2013-07-08:2013-07-08&phr : : -: : : : phrasedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : =day&goals=0&target_all=yes : &goals=0&target_all=yes : : -: : : : : 7 : : -: : : : : 3 : : -: : : : howCampStat&cid=4121349&detail=Yes&types=days : owCampStat&cid=4121349&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-07-08:2013-07-09,2013-06-01:2013-06-06,2013-04-01:2013-06-03&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : : owClients : : -: : : : 15 : : : -: : : : l=Yes&types=days&cid=7103094 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7103094&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 4 : : -: : : : 84 : : : -: : : : 89 : : : -: : : : 1&mediaType=text : : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 2 : : -: : : : 86 : : : -: : : : : 4 : : -: : : : 12 : : : -: : : : : 6 : : -: : : : : owCamps&ulogin=dc12a : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 9 : : -: : : : l=Yes&types=days&cid=141133 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=141133&sho : : -: : : : : w_banners_stat=1 : : -: : : : : 108927&cmd=showCamp : : -: : : : howCamp&cid=6886521&tab=off : owCamp&cid=6886521 : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=7133822 : : -: : : : : owCamps&ulogin=mitsubishi-galant2013 : : -: : : : ditCamp&cid=7088102 : owCamp&cid=7088102 : : -: : : : l=Yes&types=days&cid=5033617 : : : -: : : : : 2 : : -: : : : 56 : : : -: : : : : =Yes&types=days&cid=5056684 : : -: : : : 25 : : : -: : : : : owCamps : : -: : : : l=Yes&types=days&cid=6479409 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=6479409&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 9 : : -: : : : : 85.0 : : -: : : : : =Yes&types=days&cid=7133333 : : -: : : : : owCampStat&cid=3618668&detail=Yes&types=days : : -: : : : 74 : owCamp&cid=4275274&tag=303211 : : -: : : : : =Yes&types=days&cid=3298907 : : -: : : : : 9 : : -: : : : : 1 : : -: : : : howCamp&cid=5931713&tab=wait : owCamp&cid=5931713&tab=active : : -: : : : : &mediaType=text : : -: : : : 22 : : : -: : : : 002.0 : 7 : : -: : : : : e_stat=0&online_stat=0&stat_type=campdate&group=day&target_0=1&targ : : -: : : : : et_1=1 : : -: : : : : 4 : : -: : : : mportCampSuccess&cid=4202002&make_new_camp=0 : owCamp&cid=4202002 : : -: : : : : 5 : : -: : : : 87 : : : -: : : : howCamps : owCamps : : -: : : : 99 : : : -: : : : : 3 : : -: : : : 90 : 0 : : -: : : : edirlevel=1373611946.0&authredirlevel=1372140313.0&authredirlevel= : : : -: : : : 1371447906.0&authredirlevel=1368426548.0&authredirlevel=1367818443 : : : -: : : : .0&cmd=showCamps&ulogin=Newdesign-Larus : : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : 97 : : : -: : : : : 9 : : -: : : : howCamps&ulogin=ogurcova-mkt : =Yes&types=days&cid=4139609&ulogin=ogurcova-mkt : : -: : : : : 2 : : -: : : : : 2 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : : 136655&cmd=showCamp : : -: : : : : 8 : : -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -1000 tuples (4.8s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (2.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (45.218ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (44.066ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' A ND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (2.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' A ND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (44.215ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' A ND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (43.273ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -+---+-----------+ -+===+===========+ -+---+-----------+ -0 tuples (131.560ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -+---+-----------+ -+===+===========+ -+---+-----------+ -0 tuples (100.739ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -+---+-----------+ -+===+===========+ -+---+-----------+ -0 tuples (100.064ms) -sql>\q -stop time: Пт. сент. 13 11:52:17 MSK 2013 diff --git a/benchmark/monetdb/log/log_100m_corrected b/benchmark/monetdb/log/log_100m_corrected deleted file mode 100644 index 824cdd0a448..00000000000 --- a/benchmark/monetdb/log/log_100m_corrected +++ /dev/null @@ -1,1295 +0,0 @@ -start time: Tue Sep 17 12:00:01 MSK 2013 -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT count(*) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m; -+-----------+ -+===========+ -+-----------+ -1 tuple (28.886ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT count(*) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m; -+-----------+ -+===========+ -+-----------+ -1 tuple (3.581ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT count(*) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m; -+-----------+ -+===========+ -+-----------+ -1 tuple (3.829ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT count(*) FROM hits_100m WHERE AdvEngineID <> 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m WHERE AdvEngineID <> 0; -+--------+ -+========+ -+--------+ -1 tuple (2.7s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT count(*) FROM hits_100m WHERE AdvEngineID <> 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m WHERE AdvEngineID <> 0; -+--------+ -+========+ -+--------+ -1 tuple (139.637ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT count(*) FROM hits_100m WHERE AdvEngineID <> 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_100m WHERE AdvEngineID <> 0; -+--------+ -+========+ -+--------+ -1 tuple (60.264ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -+---------+-----------+--------------------------+ -+=========+===========+==========================+ -+---------+-----------+--------------------------+ -1 tuple (1.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -+---------+-----------+--------------------------+ -+=========+===========+==========================+ -+---------+-----------+--------------------------+ -1 tuple (146.556ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_100m; -+---------+-----------+--------------------------+ -+=========+===========+==========================+ -+---------+-----------+--------------------------+ -1 tuple (146.063ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT sum(UserID) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_100m; -overflow in calculation. -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT sum(UserID) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_100m; -overflow in calculation. -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT sum(UserID) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_100m; -overflow in calculation. -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT count(DISTINCT UserID) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_100m; -+----------+ -+==========+ -+----------+ -1 tuple (7.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT count(DISTINCT UserID) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_100m; -+----------+ -+==========+ -+----------+ -1 tuple (5.6s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT count(DISTINCT UserID) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_100m; -+----------+ -+==========+ -+----------+ -1 tuple (5.4s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -+---------+ -+=========+ -+---------+ -1 tuple (15.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -+---------+ -+=========+ -+---------+ -1 tuple (12.7s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_100m; -+---------+ -+=========+ -+---------+ -1 tuple (11.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT min(EventDate), max(EventDate) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_100m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (1.0s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT min(EventDate), max(EventDate) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_100m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (38.566ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT min(EventDate), max(EventDate) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_100m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (36.837ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID <> 0 GROUP 0 GROUP B Y AdvEngineID ORDER BY co < AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY cou nt(*) DESC; -+-------------+--------+ -+=============+========+ -+-------------+--------+ -18 tuples (687.604ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID <> 0 GROUP 0 GROUP B Y AdvEngineID ORDER BY co < AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY cou nt(*) DESC; -+-------------+--------+ -+=============+========+ -+-------------+--------+ -18 tuples (26.271ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_100m WHERE AdvEngineID <> 0 GROUP 0 GROUP B Y AdvEngineID ORDER BY co < AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY cou nt(*) DESC; -+-------------+--------+ -+=============+========+ -+-------------+--------+ -18 tuples (30.261ms) -sql>\q --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY Regi \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY Regi \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_100m GROUP BY Regi \q --- агрегация, среднее количество ключей.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), c \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), c \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_100m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), c \q --- агрегация, среднее количество ключей, несколько агрегатных функций.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE <, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel <> '' GR '' GRO UP BY MobilePhoneModel OR '' GROUP BY MobilePhoneModel ORD ER BY u DESC LIMIT 10; -+------------------+--------+ -+==================+========+ -+------------------+--------+ -10 tuples (22.9s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE <, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel <> '' GR '' GRO UP BY MobilePhoneModel OR '' GROUP BY MobilePhoneModel ORD ER BY u DESC LIMIT 10; -+------------------+--------+ -+==================+========+ -+------------------+--------+ -10 tuples (20.0s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE <, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel <> '' GR '' GRO UP BY MobilePhoneModel OR '' GROUP BY MobilePhoneModel ORD ER BY u DESC LIMIT 10; -+------------------+--------+ -+==================+========+ -+------------------+--------+ -10 tuples (20.8s) -sql>\q --- мощная фильтрация по строкам, затем агрегация по строкам.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hi '' GROUP BY Mobile <_100m WHERE MobilePhoneModel <> '' GROUP BY MobileP hone, MobilePhoneModel OR '' GROUP BY MobilePhone, MobilePhoneModel ORD ER BY u DESC LIMIT 10; -+-------------+------------------+--------+ -+=============+==================+========+ -+-------------+------------------+--------+ -10 tuples (20.4s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hi '' GROUP BY Mobile <_100m WHERE MobilePhoneModel <> '' GROUP BY MobileP hone, MobilePhoneModel OR '' GROUP BY MobilePhone, MobilePhoneModel ORD ER BY u DESC LIMIT 10; -+-------------+------------------+--------+ -+=============+==================+========+ -+-------------+------------------+--------+ -10 tuples (20.8s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_100m WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hi '' GROUP BY Mobile <_100m WHERE MobilePhoneModel <> '' GROUP BY MobileP hone, MobilePhoneModel OR '' GROUP BY MobilePhone, MobilePhoneModel ORD ER BY u DESC LIMIT 10; -+-------------+------------------+--------+ -+=============+==================+========+ -+-------------+------------------+--------+ -10 tuples (23.3s) -sql>\q --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase <> '' GRO '' GROU P BY SearchPhrase ORDER B '' GROUP BY SearchPhrase ORDER BY  count(*) DESC LIMIT 10; -+---------------------------------+-------+ -+=================================+=======+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase <> '' GRO '' GROU P BY SearchPhrase ORDER B '' GROUP BY SearchPhrase ORDER BY  count(*) DESC LIMIT 10; -+---------------------------------+-------+ -+=================================+=======+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase <> '' GRO '' GROU P BY SearchPhrase ORDER B '' GROUP BY SearchPhrase ORDER BY  count(*) DESC LIMIT 10; -+---------------------------------+-------+ -+=================================+=======+ -write error -sql>\q --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE Sea '' GROUP BY S '' GROUP BY Se archPhrase ORDER BY u DES '' GROUP BY SearchPhrase ORDER BY u DESC  LIMIT 10; -+---------------------------------+-------+ -+=================================+=======+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE Sea '' GROUP BY S '' GROUP BY Se archPhrase ORDER BY u DES '' GROUP BY SearchPhrase ORDER BY u DESC  LIMIT 10; -+---------------------------------+-------+ -+=================================+=======+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_100m WHERE Sea '' GROUP BY S '' GROUP BY Se archPhrase ORDER BY u DES '' GROUP BY SearchPhrase ORDER BY u DESC  LIMIT 10; -+---------------------------------+-------+ -+=================================+=======+ -write error -sql>\q --- агрегация чуть сложнее.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE Search '' GROUP BY Sear '' GROUP BY Searc hEngineID, SearchPhrase O '' GROUP BY SearchEngineID, SearchPhrase OR DER BY count(*) DESC LIMI \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE Search '' GROUP BY Sear '' GROUP BY Searc hEngineID, SearchPhrase O '' GROUP BY SearchEngineID, SearchPhrase OR DER BY count(*) DESC LIMI \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_100m WHERE Search '' GROUP BY Sear '' GROUP BY Searc hEngineID, SearchPhrase O '' GROUP BY SearchEngineID, SearchPhrase OR DER BY count(*) DESC LIMI \q --- агрегация по числу и строке, большое количество ключей.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) < FROM hits_100m GROUP BY UserID ORDER BY count(*) D ESC LIMIT 10; -+---------------------+-------+ -+=====================+=======+ -+---------------------+-------+ -10 tuples (6.8s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) < FROM hits_100m GROUP BY UserID ORDER BY count(*) D ESC LIMIT 10; -+---------------------+-------+ -+=====================+=======+ -+---------------------+-------+ -10 tuples (6.9s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_100m GROUP BY UserID ORDER BY count(*) < FROM hits_100m GROUP BY UserID ORDER BY count(*) D ESC LIMIT 10; -+---------------------+-------+ -+=====================+=======+ -+---------------------+-------+ -10 tuples (6.7s) -sql>\q --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, Sea \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, Sea \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, Sea \q --- ещё более сложная агрегация.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, Sea \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, Sea \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_100m GROUP BY UserID, Sea 4dRYFidh^hUD(f@Y$K2S>2E{0}Q!KL@@tt z9dckjHjm9SdM3<)P!}8X;!FcpT3PbEfYy92Tg+CWuhb+@4RrH3kX+low7 z11bY>3qhp=d$OSy3zDus>}egbSooGG?OvCF(eo;l!Pq0PLg_BvtG2#b23a+TCbbRu z0Lu}rE31+Mr7y@feS%7By*^zmEJ}v7E$R5!ZcR841Wsshf)FiQhN-U9mqquP#A>x` z3^YX}efn}d{;KpDr;3HmM|}bh_UWVT4`ibZ5B>%_Jq;vKWZi(?4oC-*tVr7m0epw9 zq;}}54X`}Dlns3E4n9y%Ef(~BWx@IXV82)>&<55;5DNPPqCFC!wY0o+e-pNJqBR%G zK)yHV`$7p{bzl(K(Ry82I>6T4#ER2!w*}oYw1}j&_ty6`eC;yca-bTePp9FFjkOiK zng1p5d>6jlSle2Md;j!_$7;gb#^TZjE#OmFTx#8i<>pFr69oRVb986_M)qf(JFQsw z3ms_gAip4Y-puYCX3rX9&yQH#=9@h!ciyff{oK@8rk9GfK2Q3&sj*Bib<%qYJQ#z% zLUbqz^IfRdb^O%QZy6IVm;LtFnQ&pkepw#}!Nt;_&|4ua-8JTx<`YgHWr_G-l`S%ro4c zIuDBggY&m%=o@-F^cAM(-e(-ogAZn%n`d9ktY*r|Q2hLz$0o5UN{RQx3_Le?XRgcm zAuE1JMvsgiF0G_{PyA4?agEv82t!c7_Qrf`{o;Z%>qqm%JVk(Yh!D+%XF!pdht5vT z->qlX6AHafF#QO&nyN87!;sfyr)F`rtgjf2sa9R=wXuxK(yy0b>QP$p45uJt^CroE3dcuZ_R0~=$ zgrZ|2np&#ps-QEpJH5eWKgZ^q$a0`y0g+>(dR`u(XbEXmm=+~-kAgJ$Pv_;EvZ0W$ zMg`pp+Ma@CvDw;euKGR+Lv96*y2U2pS-;Hx;D~sIqv5sn_|}xIbwQPlt!%l)JwopY z@;&FKhaZb+W3Iy4WQ;Z@PyG+Y%jn~r)sAdp|2`)6c^WmB88ZxhPKG}7)c;IspxfmK z^K(=f z?~4nP7lL@&=CBHSM4D5~Bjp|K)TCNI)o5n)$}Qw277HRI{X`(Dv<;C=Ov9Kg@w8BF zETG99h?nT8m>8kt*AZ!eo3@uJg0vgI0p@8$Lz);*CXEILq#3%YLGKZ%ka&@l7)c8~ zs)Au(Jd*g}LpSjA@KKZ3DtR{Ru-$t^np4ap&;O7!?U! za$FA`x-68MFZ}le;O9ktP84Q2FU@H#uLZrMODlYE3i|)ulJSE!sBSC0-D+R!d3V4d zN)fWdB+x{2hRar8;=!`& z-F55(`$YW_i}!Szwl3tVX_KBbpKUsJ+|~qiPt5DODBY(7Kgqj!R~sW5C*n6(NYxN0;ioqcOj%r#J)rADI6|W_d@w7OlsSw2bd1 zjrSXkF!7vOva+^hg-zwjMy*H*$7<~d>SZRa^DlNvuuW&u;1~5(p+X`ET;Xe`#-9W+ zE_`7${)C3Z2!xQpa}DNVk(G-@+#(m@)WirekrIrTCk~*_co|9!7SCf+2lW6IdGYt- zX2l4jGFC=K_EqhLz*redGki2cD8=f*axxm97qR$T>jxU&&5ko-8*msH&lebv$PBh= z#<)#TGGYAWHlGBNr$Fh6ka`N1p9sLGKUWCX&v17btj{N`PadIcxzXt5Mv+##F#d|v z_9gKRlnN~3UuHXWWjjMi9-&MsFsb0JLd#ZKMX*9dDx7bw5rms|;~XBDZ`k}JU-J*& z7!chMka@eD*V`o`lrIcA#y0;X96izWW1@KmRtgCT0j-tHQji&VxFSmdWAqD#Ny$X# zaIF}9hFKAEZ$)5i&e%Lf4Va!B{hr5+g%<(+=quuw9*)gFttSlhXAS!A)9t#Q-M*!R z-I7AoU1QGOlkZxC z?N>8qzZ$=7%P5pSe0N1B74FKzB&iT&#pV>Qzgv>cBFKtODFR|cw+gbbPOahZ8r-O{ zy?C-i8u1B5&PF9uWR6y3=B&Fu8X;2WIkxe5W?G53Nh4>Yk_izeL_AcIB@(I45AXi; z*!qd?agHt#G=wp_^3%Q0U)gbuj{#3`H*@pMdJZ&_IkR3u7mg!;l7iH{`D9VVh5c+ zI>mxUbVJe%(1~_HG!d%AXjf%bHn>jJ4U*cZ==%!RunPv8w9~tx8;XklBwV3IiBQjy zm7YqB#e2~DX4$9d1zY6`-a^Xwj+w2kd#%k4oP(4gINcyS1VdMpVSTOD+-$C`mZqxJ zDQK>On==i!p@sp8g)+!H#vW0LqLUJocvuM6v8!B|>@>Q&2uPeSyFN{!r?><~m-=l~L6=R?&8tG%;0UPyRwL?n2hr^W# GmHz{I+~t}8 diff --git a/benchmark/monetdb/log/log_100m_corrected_2 b/benchmark/monetdb/log/log_100m_corrected_2 deleted file mode 100644 index ea77a30e3cd..00000000000 --- a/benchmark/monetdb/log/log_100m_corrected_2 +++ /dev/null @@ -1,3643 +0,0 @@ -start time: Вт. сент. 17 17:33:21 MSK 2013 -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Referer <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Refer er <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -TypeException:user.s1_1[3413]:'calc.ifthenelse' undefined in: _7676:any := calc.ifthenelse(_7672:bit, A8:bte, _7674:lng) -TypeException:user.s1_1[3415]:'bat.insert' undefined in: _7679:any := bat.insert(_7667:bat[:oid,:lng], _7677:oid, _7676:any) -TypeException:user.s1_1[3423]:'calc.ifthenelse' undefined in: _7692:any := calc.ifthenelse(_7688:bit, A8:bte, _7690:lng) -TypeException:user.s1_1[3425]:'bat.insert' undefined in: _7695:any := bat.insert(_7685:bat[:oid,:lng], _7693:oid, _7692:any) -TypeException:user.s1_1[3433]:'calc.ifthenelse' undefined in: _7710:any := calc.ifthenelse(_7706:bit, A8:bte, _7708:lng) -TypeException:user.s1_1[3435]:'bat.insert' undefined in: _7713:any := bat.insert(_7701:bat[:oid,:lng], _7711:oid, _7710:any) -TypeException:user.s1_1[3443]:'calc.ifthenelse' undefined in: _7726:any := calc.ifthenelse(_7722:bit, A8:bte, _7724:lng) -TypeException:user.s1_1[3445]:'bat.insert' undefined in: _7729:any := bat.insert(_7719:bat[:oid,:lng], _7727:oid, _7726:any) -TypeException:user.s1_1[3453]:'calc.ifthenelse' undefined in: _7744:any := calc.ifthenelse(_7740:bit, A8:bte, _7742:lng) -TypeException:user.s1_1[3455]:'bat.insert' undefined in: _7747:any := bat.insert(_7735:bat[:oid,:lng], _7745:oid, _7744:any) -TypeException:user.s1_1[3463]:'calc.ifthenelse' undefined in: _7760:any := calc.ifthenelse(_7756:bit, A8:bte, _7758:lng) -TypeException:user.s1_1[3465]:'bat.insert' undefined in: _7763:any := bat.insert(_7753:bat[:oid,:lng], _7761:oid, _7760:any) -TypeException:user.s1_1[3473]:'calc.ifthenelse' undefined in: _7778:any := calc.ifthenelse(_7774:bit, A8:bte, _7776:lng) -TypeException:user.s1_1[3475]:'bat.insert' undefined in: _7781:any := bat.insert(_7769:bat[:oid,:lng], _7779:oid, _7778:any) -TypeException:user.s1_1[3483]:'calc.ifthenelse' undefined in: _7794:any := calc.ifthenelse(_7790:bit, A8:bte, _7792:lng) -TypeException:user.s1_1[3485]:'bat.insert' undefined in: _7797:any := bat.insert(_7787:bat[:oid,:lng], _7795:oid, _7794:any) -TypeException:user.s1_1[3493]:'calc.ifthenelse' undefined in: _7812:any := calc.ifthenelse(_7808:bit, A8:bte, _7810:lng) -TypeException:user.s1_1[3495]:'bat.insert' undefined in: _7815:any := bat.insert(_7803:bat[:oid,:lng], _7813:oid, _7812:any) -TypeException:user.s1_1[3503]:'calc.ifthenelse' undefined in: _7828:any := calc.ifthenelse(_7824:bit, A8:bte, _7826:lng) -TypeException:user.s1_1[3505]:'bat.insert' undefined in: _7831:any := bat.insert(_7821:bat[:oid,:lng], _7829:oid, _7828:any) -TypeException:user.s1_1[3513]:'calc.ifthenelse' undefined in: _7846:any := calc.ifthenelse(_7842:bit, A8:bte, _7844:lng) -TypeException:user.s1_1[3515]:'bat.insert' undefined in: _7849:any := bat.insert(_7837:bat[:oid,:lng], _7847:oid, _7846:any) -TypeException:user.s1_1[3523]:'calc.ifthenelse' undefined in: _7862:any := calc.ifthenelse(_7858:bit, A8:bte, _7860:lng) -TypeException:user.s1_1[3525]:'bat.insert' undefined in: _7865:any := bat.insert(_7855:bat[:oid,:lng], _7863:oid, _7862:any) -TypeException:user.s1_1[3533]:'calc.ifthenelse' undefined in: _7880:any := calc.ifthenelse(_7876:bit, A8:bte, _7878:lng) -TypeException:user.s1_1[3535]:'bat.insert' undefined in: _7883:any := bat.insert(_7871:bat[:oid,:lng], _7881:oid, _7880:any) -TypeException:user.s1_1[3543]:'calc.ifthenelse' undefined in: _7896:any := calc.ifthenelse(_7892:bit, A8:bte, _7894:lng) -TypeException:user.s1_1[3545]:'bat.insert' undefined in: _7899:any := bat.insert(_7889:bat[:oid,:lng], _7897:oid, _7896:any) -TypeException:user.s1_1[3553]:'calc.ifthenelse' undefined in: _7914:any := calc.ifthenelse(_7910:bit, A8:bte, _7912:lng) -TypeException:user.s1_1[3555]:'bat.insert' undefined in: _7917:any := bat.insert(_7905:bat[:oid,:lng], _7915:oid, _7914:any) -TypeException:user.s1_1[3563]:'calc.ifthenelse' undefined in: _7930:any := calc.ifthenelse(_7926:bit, A8:bte, _7928:lng) -TypeException:user.s1_1[3565]:'bat.insert' undefined in: _7933:any := bat.insert(_7923:bat[:oid,:lng], _7931:oid, _7930:any) -TypeException:user.s1_1[3573]:'calc.ifthenelse' undefined in: _7948:any := calc.ifthenelse(_7944:bit, A8:bte, _7946:lng) -TypeException:user.s1_1[3575]:'bat.insert' undefined in: _7951:any := bat.insert(_7939:bat[:oid,:lng], _7949:oid, _7948:any) -TypeException:user.s1_1[3583]:'calc.ifthenelse' undefined in: _7964:any := calc.ifthenelse(_7960:bit, A8:bte, _7962:lng) -TypeException:user.s1_1[3585]:'bat.insert' undefined in: _7967:any := bat.insert(_7957:bat[:oid,:lng], _7965:oid, _7964:any) -TypeException:user.s1_1[3593]:'calc.ifthenelse' undefined in: _7982:any := calc.ifthenelse(_7978:bit, A8:bte, _7980:lng) -TypeException:user.s1_1[3595]:'bat.insert' undefined in: _7985:any := bat.insert(_7973:bat[:oid,:lng], _7983:oid, _7982:any) -TypeException:user.s1_1[3603]:'calc.ifthenelse' undefined in: _7998:any := calc.ifthenelse(_7994:bit, A8:bte, _7996:lng) -TypeException:user.s1_1[3605]:'bat.insert' undefined in: _8001:any := bat.insert(_7991:bat[:oid,:lng], _7999:oid, _7998:any) -TypeException:user.s1_1[3613]:'calc.ifthenelse' undefined in: _8016:any := calc.ifthenelse(_8012:bit, A8:bte, _8014:lng) -TypeException:user.s1_1[3615]:'bat.insert' undefined in: _8019:any := bat.insert(_8007:bat[:oid,:lng], _8017:oid, _8016:any) -TypeException:user.s1_1[3623]:'calc.ifthenelse' undefined in: _8032:any := calc.ifthenelse(_8028:bit, A8:bte, _8030:lng) -TypeException:user.s1_1[3625]:'bat.insert' undefined in: _8035:any := bat.insert(_8025:bat[:oid,:lng], _8033:oid, _8032:any) -TypeException:user.s1_1[3633]:'calc.ifthenelse' undefined in: _8050:any := calc.ifthenelse(_8046:bit, A8:bte, _8048:lng) -TypeException:user.s1_1[3635]:'bat.insert' undefined in: _8053:any := bat.insert(_8041:bat[:oid,:lng], _8051:oid, _8050:any) -TypeException:user.s1_1[3643]:'calc.ifthenelse' undefined in: _8066:any := calc.ifthenelse(_8062:bit, A8:bte, _8064:lng) -TypeException:user.s1_1[3645]:'bat.insert' undefined in: _8069:any := bat.insert(_8059:bat[:oid,:lng], _8067:oid, _8066:any) -TypeException:user.s1_1[3653]:'calc.ifthenelse' undefined in: _8084:any := calc.ifthenelse(_8080:bit, A8:bte, _8082:lng) -TypeException:user.s1_1[3655]:'bat.insert' undefined in: _8087:any := bat.insert(_8075:bat[:oid,:lng], _8085:oid, _8084:any) -TypeException:user.s1_1[3663]:'calc.ifthenelse' undefined in: _8100:any := calc.ifthenelse(_8096:bit, A8:bte, _8098:lng) -TypeException:user.s1_1[3665]:'bat.insert' undefined in: _8103:any := bat.insert(_8093:bat[:oid,:lng], _8101:oid, _8100:any) -TypeException:user.s1_1[3673]:'calc.ifthenelse' undefined in: _8118:any := calc.ifthenelse(_8114:bit, A8:bte, _8116:lng) -TypeException:user.s1_1[3675]:'bat.insert' undefined in: _8121:any := bat.insert(_8109:bat[:oid,:lng], _8119:oid, _8118:any) -TypeException:user.s1_1[3683]:'calc.ifthenelse' undefined in: _8134:any := calc.ifthenelse(_8130:bit, A8:bte, _8132:lng) -TypeException:user.s1_1[3685]:'bat.insert' undefined in: _8137:any := bat.insert(_8127:bat[:oid,:lng], _8135:oid, _8134:any) -TypeException:user.s1_1[3693]:'calc.ifthenelse' undefined in: _8152:any := calc.ifthenelse(_8148:bit, A8:bte, _8150:lng) -TypeException:user.s1_1[3695]:'bat.insert' undefined in: _8155:any := bat.insert(_8143:bat[:oid,:lng], _8153:oid, _8152:any) -TypeException:user.s1_1[3703]:'calc.ifthenelse' undefined in: _8168:any := calc.ifthenelse(_8164:bit, A8:bte, _8166:lng) -TypeException:user.s1_1[3705]:'bat.insert' undefined in: _8171:any := bat.insert(_8161:bat[:oid,:lng], _8169:oid, _8168:any) -TypeException:user.s1_1[3713]:'calc.ifthenelse' undefined in: _8186:any := calc.ifthenelse(_8182:bit, A8:bte, _8184:lng) -TypeException:user.s1_1[3715]:'bat.insert' undefined in: _8189:any := bat.insert(_8177:bat[:oid,:lng], _8187:oid, _8186:any) -TypeException:user.s1_1[3723]:'calc.ifthenelse' undefined in: _8202:any := calc.ifthenelse(_8198:bit, A8:bte, _8200:lng) -TypeException:user.s1_1[3725]:'bat.insert' undefined in: _8205:any := bat.insert(_8195:bat[:oid,:lng], _8203:oid, _8202:any) -program contains errors -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Referer <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Refer er <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -TypeException:user.s1_1[3413]:'calc.ifthenelse' undefined in: _7676:any := calc.ifthenelse(_7672:bit, A8:bte, _7674:lng) -TypeException:user.s1_1[3415]:'bat.insert' undefined in: _7679:any := bat.insert(_7667:bat[:oid,:lng], _7677:oid, _7676:any) -TypeException:user.s1_1[3423]:'calc.ifthenelse' undefined in: _7692:any := calc.ifthenelse(_7688:bit, A8:bte, _7690:lng) -TypeException:user.s1_1[3425]:'bat.insert' undefined in: _7695:any := bat.insert(_7685:bat[:oid,:lng], _7693:oid, _7692:any) -TypeException:user.s1_1[3433]:'calc.ifthenelse' undefined in: _7710:any := calc.ifthenelse(_7706:bit, A8:bte, _7708:lng) -TypeException:user.s1_1[3435]:'bat.insert' undefined in: _7713:any := bat.insert(_7701:bat[:oid,:lng], _7711:oid, _7710:any) -TypeException:user.s1_1[3443]:'calc.ifthenelse' undefined in: _7726:any := calc.ifthenelse(_7722:bit, A8:bte, _7724:lng) -TypeException:user.s1_1[3445]:'bat.insert' undefined in: _7729:any := bat.insert(_7719:bat[:oid,:lng], _7727:oid, _7726:any) -TypeException:user.s1_1[3453]:'calc.ifthenelse' undefined in: _7744:any := calc.ifthenelse(_7740:bit, A8:bte, _7742:lng) -TypeException:user.s1_1[3455]:'bat.insert' undefined in: _7747:any := bat.insert(_7735:bat[:oid,:lng], _7745:oid, _7744:any) -TypeException:user.s1_1[3463]:'calc.ifthenelse' undefined in: _7760:any := calc.ifthenelse(_7756:bit, A8:bte, _7758:lng) -TypeException:user.s1_1[3465]:'bat.insert' undefined in: _7763:any := bat.insert(_7753:bat[:oid,:lng], _7761:oid, _7760:any) -TypeException:user.s1_1[3473]:'calc.ifthenelse' undefined in: _7778:any := calc.ifthenelse(_7774:bit, A8:bte, _7776:lng) -TypeException:user.s1_1[3475]:'bat.insert' undefined in: _7781:any := bat.insert(_7769:bat[:oid,:lng], _7779:oid, _7778:any) -TypeException:user.s1_1[3483]:'calc.ifthenelse' undefined in: _7794:any := calc.ifthenelse(_7790:bit, A8:bte, _7792:lng) -TypeException:user.s1_1[3485]:'bat.insert' undefined in: _7797:any := bat.insert(_7787:bat[:oid,:lng], _7795:oid, _7794:any) -TypeException:user.s1_1[3493]:'calc.ifthenelse' undefined in: _7812:any := calc.ifthenelse(_7808:bit, A8:bte, _7810:lng) -TypeException:user.s1_1[3495]:'bat.insert' undefined in: _7815:any := bat.insert(_7803:bat[:oid,:lng], _7813:oid, _7812:any) -TypeException:user.s1_1[3503]:'calc.ifthenelse' undefined in: _7828:any := calc.ifthenelse(_7824:bit, A8:bte, _7826:lng) -TypeException:user.s1_1[3505]:'bat.insert' undefined in: _7831:any := bat.insert(_7821:bat[:oid,:lng], _7829:oid, _7828:any) -TypeException:user.s1_1[3513]:'calc.ifthenelse' undefined in: _7846:any := calc.ifthenelse(_7842:bit, A8:bte, _7844:lng) -TypeException:user.s1_1[3515]:'bat.insert' undefined in: _7849:any := bat.insert(_7837:bat[:oid,:lng], _7847:oid, _7846:any) -TypeException:user.s1_1[3523]:'calc.ifthenelse' undefined in: _7862:any := calc.ifthenelse(_7858:bit, A8:bte, _7860:lng) -TypeException:user.s1_1[3525]:'bat.insert' undefined in: _7865:any := bat.insert(_7855:bat[:oid,:lng], _7863:oid, _7862:any) -TypeException:user.s1_1[3533]:'calc.ifthenelse' undefined in: _7880:any := calc.ifthenelse(_7876:bit, A8:bte, _7878:lng) -TypeException:user.s1_1[3535]:'bat.insert' undefined in: _7883:any := bat.insert(_7871:bat[:oid,:lng], _7881:oid, _7880:any) -TypeException:user.s1_1[3543]:'calc.ifthenelse' undefined in: _7896:any := calc.ifthenelse(_7892:bit, A8:bte, _7894:lng) -TypeException:user.s1_1[3545]:'bat.insert' undefined in: _7899:any := bat.insert(_7889:bat[:oid,:lng], _7897:oid, _7896:any) -TypeException:user.s1_1[3553]:'calc.ifthenelse' undefined in: _7914:any := calc.ifthenelse(_7910:bit, A8:bte, _7912:lng) -TypeException:user.s1_1[3555]:'bat.insert' undefined in: _7917:any := bat.insert(_7905:bat[:oid,:lng], _7915:oid, _7914:any) -TypeException:user.s1_1[3563]:'calc.ifthenelse' undefined in: _7930:any := calc.ifthenelse(_7926:bit, A8:bte, _7928:lng) -TypeException:user.s1_1[3565]:'bat.insert' undefined in: _7933:any := bat.insert(_7923:bat[:oid,:lng], _7931:oid, _7930:any) -TypeException:user.s1_1[3573]:'calc.ifthenelse' undefined in: _7948:any := calc.ifthenelse(_7944:bit, A8:bte, _7946:lng) -TypeException:user.s1_1[3575]:'bat.insert' undefined in: _7951:any := bat.insert(_7939:bat[:oid,:lng], _7949:oid, _7948:any) -TypeException:user.s1_1[3583]:'calc.ifthenelse' undefined in: _7964:any := calc.ifthenelse(_7960:bit, A8:bte, _7962:lng) -TypeException:user.s1_1[3585]:'bat.insert' undefined in: _7967:any := bat.insert(_7957:bat[:oid,:lng], _7965:oid, _7964:any) -TypeException:user.s1_1[3593]:'calc.ifthenelse' undefined in: _7982:any := calc.ifthenelse(_7978:bit, A8:bte, _7980:lng) -TypeException:user.s1_1[3595]:'bat.insert' undefined in: _7985:any := bat.insert(_7973:bat[:oid,:lng], _7983:oid, _7982:any) -TypeException:user.s1_1[3603]:'calc.ifthenelse' undefined in: _7998:any := calc.ifthenelse(_7994:bit, A8:bte, _7996:lng) -TypeException:user.s1_1[3605]:'bat.insert' undefined in: _8001:any := bat.insert(_7991:bat[:oid,:lng], _7999:oid, _7998:any) -TypeException:user.s1_1[3613]:'calc.ifthenelse' undefined in: _8016:any := calc.ifthenelse(_8012:bit, A8:bte, _8014:lng) -TypeException:user.s1_1[3615]:'bat.insert' undefined in: _8019:any := bat.insert(_8007:bat[:oid,:lng], _8017:oid, _8016:any) -TypeException:user.s1_1[3623]:'calc.ifthenelse' undefined in: _8032:any := calc.ifthenelse(_8028:bit, A8:bte, _8030:lng) -TypeException:user.s1_1[3625]:'bat.insert' undefined in: _8035:any := bat.insert(_8025:bat[:oid,:lng], _8033:oid, _8032:any) -TypeException:user.s1_1[3633]:'calc.ifthenelse' undefined in: _8050:any := calc.ifthenelse(_8046:bit, A8:bte, _8048:lng) -TypeException:user.s1_1[3635]:'bat.insert' undefined in: _8053:any := bat.insert(_8041:bat[:oid,:lng], _8051:oid, _8050:any) -TypeException:user.s1_1[3643]:'calc.ifthenelse' undefined in: _8066:any := calc.ifthenelse(_8062:bit, A8:bte, _8064:lng) -TypeException:user.s1_1[3645]:'bat.insert' undefined in: _8069:any := bat.insert(_8059:bat[:oid,:lng], _8067:oid, _8066:any) -TypeException:user.s1_1[3653]:'calc.ifthenelse' undefined in: _8084:any := calc.ifthenelse(_8080:bit, A8:bte, _8082:lng) -TypeException:user.s1_1[3655]:'bat.insert' undefined in: _8087:any := bat.insert(_8075:bat[:oid,:lng], _8085:oid, _8084:any) -TypeException:user.s1_1[3663]:'calc.ifthenelse' undefined in: _8100:any := calc.ifthenelse(_8096:bit, A8:bte, _8098:lng) -TypeException:user.s1_1[3665]:'bat.insert' undefined in: _8103:any := bat.insert(_8093:bat[:oid,:lng], _8101:oid, _8100:any) -TypeException:user.s1_1[3673]:'calc.ifthenelse' undefined in: _8118:any := calc.ifthenelse(_8114:bit, A8:bte, _8116:lng) -TypeException:user.s1_1[3675]:'bat.insert' undefined in: _8121:any := bat.insert(_8109:bat[:oid,:lng], _8119:oid, _8118:any) -TypeException:user.s1_1[3683]:'calc.ifthenelse' undefined in: _8134:any := calc.ifthenelse(_8130:bit, A8:bte, _8132:lng) -TypeException:user.s1_1[3685]:'bat.insert' undefined in: _8137:any := bat.insert(_8127:bat[:oid,:lng], _8135:oid, _8134:any) -TypeException:user.s1_1[3693]:'calc.ifthenelse' undefined in: _8152:any := calc.ifthenelse(_8148:bit, A8:bte, _8150:lng) -TypeException:user.s1_1[3695]:'bat.insert' undefined in: _8155:any := bat.insert(_8143:bat[:oid,:lng], _8153:oid, _8152:any) -TypeException:user.s1_1[3703]:'calc.ifthenelse' undefined in: _8168:any := calc.ifthenelse(_8164:bit, A8:bte, _8166:lng) -TypeException:user.s1_1[3705]:'bat.insert' undefined in: _8171:any := bat.insert(_8161:bat[:oid,:lng], _8169:oid, _8168:any) -TypeException:user.s1_1[3713]:'calc.ifthenelse' undefined in: _8186:any := calc.ifthenelse(_8182:bit, A8:bte, _8184:lng) -TypeException:user.s1_1[3715]:'bat.insert' undefined in: _8189:any := bat.insert(_8177:bat[:oid,:lng], _8187:oid, _8186:any) -TypeException:user.s1_1[3723]:'calc.ifthenelse' undefined in: _8202:any := calc.ifthenelse(_8198:bit, A8:bte, _8200:lng) -TypeException:user.s1_1[3725]:'bat.insert' undefined in: _8205:any := bat.insert(_8195:bat[:oid,:lng], _8203:oid, _8202:any) -program contains errors -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Referer <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_100m WHERE Refer er <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -TypeException:user.s1_1[3413]:'calc.ifthenelse' undefined in: _7676:any := calc.ifthenelse(_7672:bit, A8:bte, _7674:lng) -TypeException:user.s1_1[3415]:'bat.insert' undefined in: _7679:any := bat.insert(_7667:bat[:oid,:lng], _7677:oid, _7676:any) -TypeException:user.s1_1[3423]:'calc.ifthenelse' undefined in: _7692:any := calc.ifthenelse(_7688:bit, A8:bte, _7690:lng) -TypeException:user.s1_1[3425]:'bat.insert' undefined in: _7695:any := bat.insert(_7685:bat[:oid,:lng], _7693:oid, _7692:any) -TypeException:user.s1_1[3433]:'calc.ifthenelse' undefined in: _7710:any := calc.ifthenelse(_7706:bit, A8:bte, _7708:lng) -TypeException:user.s1_1[3435]:'bat.insert' undefined in: _7713:any := bat.insert(_7701:bat[:oid,:lng], _7711:oid, _7710:any) -TypeException:user.s1_1[3443]:'calc.ifthenelse' undefined in: _7726:any := calc.ifthenelse(_7722:bit, A8:bte, _7724:lng) -TypeException:user.s1_1[3445]:'bat.insert' undefined in: _7729:any := bat.insert(_7719:bat[:oid,:lng], _7727:oid, _7726:any) -TypeException:user.s1_1[3453]:'calc.ifthenelse' undefined in: _7744:any := calc.ifthenelse(_7740:bit, A8:bte, _7742:lng) -TypeException:user.s1_1[3455]:'bat.insert' undefined in: _7747:any := bat.insert(_7735:bat[:oid,:lng], _7745:oid, _7744:any) -TypeException:user.s1_1[3463]:'calc.ifthenelse' undefined in: _7760:any := calc.ifthenelse(_7756:bit, A8:bte, _7758:lng) -TypeException:user.s1_1[3465]:'bat.insert' undefined in: _7763:any := bat.insert(_7753:bat[:oid,:lng], _7761:oid, _7760:any) -TypeException:user.s1_1[3473]:'calc.ifthenelse' undefined in: _7778:any := calc.ifthenelse(_7774:bit, A8:bte, _7776:lng) -TypeException:user.s1_1[3475]:'bat.insert' undefined in: _7781:any := bat.insert(_7769:bat[:oid,:lng], _7779:oid, _7778:any) -TypeException:user.s1_1[3483]:'calc.ifthenelse' undefined in: _7794:any := calc.ifthenelse(_7790:bit, A8:bte, _7792:lng) -TypeException:user.s1_1[3485]:'bat.insert' undefined in: _7797:any := bat.insert(_7787:bat[:oid,:lng], _7795:oid, _7794:any) -TypeException:user.s1_1[3493]:'calc.ifthenelse' undefined in: _7812:any := calc.ifthenelse(_7808:bit, A8:bte, _7810:lng) -TypeException:user.s1_1[3495]:'bat.insert' undefined in: _7815:any := bat.insert(_7803:bat[:oid,:lng], _7813:oid, _7812:any) -TypeException:user.s1_1[3503]:'calc.ifthenelse' undefined in: _7828:any := calc.ifthenelse(_7824:bit, A8:bte, _7826:lng) -TypeException:user.s1_1[3505]:'bat.insert' undefined in: _7831:any := bat.insert(_7821:bat[:oid,:lng], _7829:oid, _7828:any) -TypeException:user.s1_1[3513]:'calc.ifthenelse' undefined in: _7846:any := calc.ifthenelse(_7842:bit, A8:bte, _7844:lng) -TypeException:user.s1_1[3515]:'bat.insert' undefined in: _7849:any := bat.insert(_7837:bat[:oid,:lng], _7847:oid, _7846:any) -TypeException:user.s1_1[3523]:'calc.ifthenelse' undefined in: _7862:any := calc.ifthenelse(_7858:bit, A8:bte, _7860:lng) -TypeException:user.s1_1[3525]:'bat.insert' undefined in: _7865:any := bat.insert(_7855:bat[:oid,:lng], _7863:oid, _7862:any) -TypeException:user.s1_1[3533]:'calc.ifthenelse' undefined in: _7880:any := calc.ifthenelse(_7876:bit, A8:bte, _7878:lng) -TypeException:user.s1_1[3535]:'bat.insert' undefined in: _7883:any := bat.insert(_7871:bat[:oid,:lng], _7881:oid, _7880:any) -TypeException:user.s1_1[3543]:'calc.ifthenelse' undefined in: _7896:any := calc.ifthenelse(_7892:bit, A8:bte, _7894:lng) -TypeException:user.s1_1[3545]:'bat.insert' undefined in: _7899:any := bat.insert(_7889:bat[:oid,:lng], _7897:oid, _7896:any) -TypeException:user.s1_1[3553]:'calc.ifthenelse' undefined in: _7914:any := calc.ifthenelse(_7910:bit, A8:bte, _7912:lng) -TypeException:user.s1_1[3555]:'bat.insert' undefined in: _7917:any := bat.insert(_7905:bat[:oid,:lng], _7915:oid, _7914:any) -TypeException:user.s1_1[3563]:'calc.ifthenelse' undefined in: _7930:any := calc.ifthenelse(_7926:bit, A8:bte, _7928:lng) -TypeException:user.s1_1[3565]:'bat.insert' undefined in: _7933:any := bat.insert(_7923:bat[:oid,:lng], _7931:oid, _7930:any) -TypeException:user.s1_1[3573]:'calc.ifthenelse' undefined in: _7948:any := calc.ifthenelse(_7944:bit, A8:bte, _7946:lng) -TypeException:user.s1_1[3575]:'bat.insert' undefined in: _7951:any := bat.insert(_7939:bat[:oid,:lng], _7949:oid, _7948:any) -TypeException:user.s1_1[3583]:'calc.ifthenelse' undefined in: _7964:any := calc.ifthenelse(_7960:bit, A8:bte, _7962:lng) -TypeException:user.s1_1[3585]:'bat.insert' undefined in: _7967:any := bat.insert(_7957:bat[:oid,:lng], _7965:oid, _7964:any) -TypeException:user.s1_1[3593]:'calc.ifthenelse' undefined in: _7982:any := calc.ifthenelse(_7978:bit, A8:bte, _7980:lng) -TypeException:user.s1_1[3595]:'bat.insert' undefined in: _7985:any := bat.insert(_7973:bat[:oid,:lng], _7983:oid, _7982:any) -TypeException:user.s1_1[3603]:'calc.ifthenelse' undefined in: _7998:any := calc.ifthenelse(_7994:bit, A8:bte, _7996:lng) -TypeException:user.s1_1[3605]:'bat.insert' undefined in: _8001:any := bat.insert(_7991:bat[:oid,:lng], _7999:oid, _7998:any) -TypeException:user.s1_1[3613]:'calc.ifthenelse' undefined in: _8016:any := calc.ifthenelse(_8012:bit, A8:bte, _8014:lng) -TypeException:user.s1_1[3615]:'bat.insert' undefined in: _8019:any := bat.insert(_8007:bat[:oid,:lng], _8017:oid, _8016:any) -TypeException:user.s1_1[3623]:'calc.ifthenelse' undefined in: _8032:any := calc.ifthenelse(_8028:bit, A8:bte, _8030:lng) -TypeException:user.s1_1[3625]:'bat.insert' undefined in: _8035:any := bat.insert(_8025:bat[:oid,:lng], _8033:oid, _8032:any) -TypeException:user.s1_1[3633]:'calc.ifthenelse' undefined in: _8050:any := calc.ifthenelse(_8046:bit, A8:bte, _8048:lng) -TypeException:user.s1_1[3635]:'bat.insert' undefined in: _8053:any := bat.insert(_8041:bat[:oid,:lng], _8051:oid, _8050:any) -TypeException:user.s1_1[3643]:'calc.ifthenelse' undefined in: _8066:any := calc.ifthenelse(_8062:bit, A8:bte, _8064:lng) -TypeException:user.s1_1[3645]:'bat.insert' undefined in: _8069:any := bat.insert(_8059:bat[:oid,:lng], _8067:oid, _8066:any) -TypeException:user.s1_1[3653]:'calc.ifthenelse' undefined in: _8084:any := calc.ifthenelse(_8080:bit, A8:bte, _8082:lng) -TypeException:user.s1_1[3655]:'bat.insert' undefined in: _8087:any := bat.insert(_8075:bat[:oid,:lng], _8085:oid, _8084:any) -TypeException:user.s1_1[3663]:'calc.ifthenelse' undefined in: _8100:any := calc.ifthenelse(_8096:bit, A8:bte, _8098:lng) -TypeException:user.s1_1[3665]:'bat.insert' undefined in: _8103:any := bat.insert(_8093:bat[:oid,:lng], _8101:oid, _8100:any) -TypeException:user.s1_1[3673]:'calc.ifthenelse' undefined in: _8118:any := calc.ifthenelse(_8114:bit, A8:bte, _8116:lng) -TypeException:user.s1_1[3675]:'bat.insert' undefined in: _8121:any := bat.insert(_8109:bat[:oid,:lng], _8119:oid, _8118:any) -TypeException:user.s1_1[3683]:'calc.ifthenelse' undefined in: _8134:any := calc.ifthenelse(_8130:bit, A8:bte, _8132:lng) -TypeException:user.s1_1[3685]:'bat.insert' undefined in: _8137:any := bat.insert(_8127:bat[:oid,:lng], _8135:oid, _8134:any) -TypeException:user.s1_1[3693]:'calc.ifthenelse' undefined in: _8152:any := calc.ifthenelse(_8148:bit, A8:bte, _8150:lng) -TypeException:user.s1_1[3695]:'bat.insert' undefined in: _8155:any := bat.insert(_8143:bat[:oid,:lng], _8153:oid, _8152:any) -TypeException:user.s1_1[3703]:'calc.ifthenelse' undefined in: _8168:any := calc.ifthenelse(_8164:bit, A8:bte, _8166:lng) -TypeException:user.s1_1[3705]:'bat.insert' undefined in: _8171:any := bat.insert(_8161:bat[:oid,:lng], _8169:oid, _8168:any) -TypeException:user.s1_1[3713]:'calc.ifthenelse' undefined in: _8186:any := calc.ifthenelse(_8182:bit, A8:bte, _8184:lng) -TypeException:user.s1_1[3715]:'bat.insert' undefined in: _8189:any := bat.insert(_8177:bat[:oid,:lng], _8187:oid, _8186:any) -TypeException:user.s1_1[3723]:'calc.ifthenelse' undefined in: _8202:any := calc.ifthenelse(_8198:bit, A8:bte, _8200:lng) -TypeException:user.s1_1[3725]:'bat.insert' undefined in: _8205:any := bat.insert(_8195:bat[:oid,:lng], _8203:oid, _8202:any) -program contains errors -sql>\q --- то же самое, но с разбивкой по доменам.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(Re solutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWid th + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18 ), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum( ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(Resolut ionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidt h + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43) , sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(R esolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(Resoluti onWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(Re solutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(Resolutio nWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_100m; -+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+ -+==============+==============+==============+==============+==============+==============+==============+==============+==============+==============+==============+ -+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+ -1 tuple (12.7s) !79 columns dropped! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(Re solutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWid th + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18 ), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum( ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(Resolut ionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidt h + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43) , sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(R esolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(Resoluti onWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(Re solutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(Resolutio nWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_100m; -+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+ -+==============+==============+==============+==============+==============+==============+==============+==============+==============+==============+==============+ -+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+ -1 tuple (11.2s) !79 columns dropped! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_100m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(Re solutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWid th + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18 ), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum( ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(Resolut ionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidt h + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43) , sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(R esolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(Resoluti onWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(Re solutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(Resolutio nWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_100m; -+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+ -+==============+==============+==============+==============+==============+==============+==============+==============+==============+==============+==============+ -+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+--------------+ -1 tuple (11.4s) !79 columns dropped! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q --- много тупых агрегатных функций.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+----------------+------------+------+------+--------------------------+ -+================+============+======+======+==========================+ -+----------------+------------+------+------+--------------------------+ -10 tuples (6m 32s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+----------------+------------+------+------+--------------------------+ -+================+============+======+======+==========================+ -+----------------+------------+------+------+--------------------------+ -10 tuples (5m 8s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+----------------+------------+------+------+--------------------------+ -+================+============+======+======+==========================+ -+----------------+------------+------+------+--------------------------+ -10 tuples (6m 10s) -sql>\q --- сложная агрегация, для больших таблиц может не хватить оперативки.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (3.2s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (1.8s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (1.8s) -sql>\q --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (41.9s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (37.0s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_100m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (40.7s) -sql>\q --- то же самое, но ещё и без фильтрации.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -\q -Connection terminated -monetdbd is not running. -spawn mclient -u monetdb -d hits -password: -could not connect to localhost:50000: Connection refused -restart server: /etc/init.d/monetdb5-sql restart -monetdbd not running. -Starting MonetDB SQL server: monetdbd. - -times: 3 -query: SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_100m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -Connection terminated --- агрегация по URL.; - -monetdbd is not running. -spawn mclient -u monetdb -d hits -password: -could not connect to localhost:50000: Connection refused -restart server: /etc/init.d/monetdb5-sql restart -monetdbd not running. -Starting MonetDB SQL server: monetdbd. - -times: 1 -query: SELECT 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected sqlINT in: "select 1, url, count(*) from hits_100m group by 1" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected sqlINT in: "select 1, url, count(*) from hits_100m group by 1" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT 1, URL, count(*) FROM hits_100m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected sqlINT in: "select 1, url, count(*) from hits_100m group by 1" -sql>\q --- агрегация по URL и числу.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_100m group by clientip, clientip -" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_100m group by clientip, clientip -" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_100m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_100m group by clientip, clientip -" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------+-----------+ -+=================================================================================+===========+ -+---------------------------------------------------------------------------------+-----------+ -10 tuples (6.4s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------+-----------+ -+=================================================================================+===========+ -+---------------------------------------------------------------------------------+-----------+ -10 tuples (2.3s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------+-----------+ -+=================================================================================+===========+ -+---------------------------------------------------------------------------------+-----------+ -10 tuples (2.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Ref resh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+------------------------------------------------+-----------+ -+================================================+===========+ -+------------------------------------------------+-----------+ -10 tuples (3.6s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Ref resh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+------------------------------------------------+-----------+ -+================================================+===========+ -+------------------------------------------------+-----------+ -10 tuples (1.0s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Ref resh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+------------------------------------------------+-----------+ -+================================================+===========+ -+------------------------------------------------+-----------+ -10 tuples (1.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+----------------------------------------------------------------------------------------------------------------------------------------------------------------+------+ -: : view : -: : s : -+================================================================================================================================================================+======+ -: tegoryID=657&search_name=&search_price_from=0&search_price_to=292381¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_25=0¶mRangeTo_25=10¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 0%26price%3D0;8000%26price_from%3D0%26price_to%3D8000%26category[]%3D224 : : -: mest : : -: tegoryID=710&search_name=&search_price_from=0&search_price_to=799053¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000 : : -: 0,2013-07-01:2013-07-09;stat_type=campdate;y1=2013;m1=07;d1=01;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: tegoryID=657&search_name=&search_price_from=0&search_price_to=292381¶m_27_33=yep¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo : : -: _16=5000¶mRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_25=0¶mRangeTo_25=10¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 01:2013-07-14&m2=07&stat_type=custom&onpage=100&group_by=phrase, retargeting&d1=08&target_1=0&cmd=showCampStat&ulogin=kuncevo-ru&reverse=ARRAY(0x17e9c4c0)&tar : : -: get_all=1&online_stat=0&m1=07&cid=4601559&goals=0&offline_stat=0&y2=13 : : -: ype}&pos={position}&key={keyword} : : -: _type_id=1&product_type_1_model_kia[]=Spectra : : -: duct_type_id=1&product_type_1_model_hyundai[]=Sonata : : -: duct_type_id=1&product_type_1_model_hyundai[]=SantaFe : : -: yundai_comp=find_in_set_any&product_type_id=1&product_type_1_model_hyundai[]=Tucson : : -: %3Dpopularity_desc%26num%3D100%26price_from%3D8000%26price_to%3D11000%26price%3D8000;11000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D100%26price_from%3D24000%26price_to%3D27000%26price%3D24000;27000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D20%26price_from%3D15000%26price_to%3D17000%26price%3D15000;17000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D20%26price_from%3D0%26price_to%3D15000%26price%3D0;15000%26category[]%3D224%26data_form%3DY : : -: 01:2013-07-14&m2=07&stat_type=custom&onpage=10000&group_by=phrase&d1=01&target_1=0&cmd=showCampStat&ulogin=UBRiR66&reverse=1&target_all=1&sort=sum&m1=07&onlin : : -: e_stat=0&cid=6715451&goals=0&y2=13&offline_stat=0 : : -: 1,2013-06-01:2013-07-11;ulogin=pb-context;stat_type=campdate;y1=2013;m1=07;d1=11;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: n=Прочитайте истории успеха клие : : -: 13-06-10:2013-07-14&stat_type=custom&group=week&onpage=10000&group_by=phrase, date&d1=08&target_1=0&cmd=showCampStat&ulogin=izosfera2012&reverse=ARRAY(0x19231 : : -: 210)&target_all=1&online_stat=0&m1=07&cid=5791188&goals=0&offline_stat=0&y2=13 : : -: 013-05-31&m2=07&group=week&d1=15&cmd=showCampStat&reverse=1&detail=yes&target_all=yes&sort=sum&online_stat=1&m1=07&cid=6744826&goals=0&y2=13&offline_stat=0 : : -: 013-04-30&m2=07&group=day&d1=15&cmd=showCampStat&ulogin=yar-tehnostroy&detail=yes&target_all=yes&online_stat=1&m1=06&cid=3539794&goals=0&offline_stat=0&y2=13 : : -: 2,2013-07-08:2013-07-14;stat_type=campdate;y1=2013;m1=06;d1=15;y2=2013;m2=07;d2=15;group=day;target_all=1&xls=1 : : -: sc%26num%3D20%26price%3D0;39000%26price_from%3D0%26price_to%3D39000%26category[]%3D1732 : : -: 02:2013-07-14&m2=07&stat_type=custom&group=day&onpage=100&group_by=date&d1=14&target_1=0&cmd=showCampStat&ulogin=aaamotors2009&reverse=ARRAY(0x4029e50)&target : : -: _all=1&online_stat=1&m1=06&cid=5300235&goals=0&offline_stat=0&y2=13 : : -: tegoryID=658&search_name=&search_price_from=0&search_price_to=845243¶m_27_40=yep¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo : : -: _16=5000¶mRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_22=1¶mRangeTo_22=1500 : : -: ,2013-07-01:2013-07-07&m2=07&group=day&d1=08&cmd=showCampStat&ulogin=conrays-mos&phrasedate=yes&online_stat=0&m1=07&cid=5073455&goals=0&offline_stat=0&y2=13 : : -: 4,2013-07-01:2013-07-15;stat_type=campdate;y1=2013;m1=05;d1=01;y2=2013;m2=05;d2=31;group=month;target_all=1&xls=1 : : -: 10:2013-07-14&m2=07&stat_type=custom&group=week&onpage=10000&group_by=phrase, date&d1=08&cmd=showCampStat&target_1=0&ulogin=avtogermes2013&reverse=1&sort=sum& : : -: target_all=1&online_stat=0&m1=07&cid=5767794&goals=0&y2=13&offline_stat=0 : : -: 7,2013-06-24:2013-06-30;stat_type=campdate;y1=2013;m1=07;d1=08;y2=2013;m2=07;d2=14;group=week;target_all=1&xls=1 : : -: 89779;cid=5688769;cid=5846786;cid=5846794;cid=5846798;cid=5846824;cid=5846814;cid=5846776;cid=5689791;cid=5902210;cid=5843727;cid=5945010;cid=7085900;cid=7086 : : -: 434;cid=7086465;cid=7086475;cid=7086483;cid=7086497;cid=7086504;cid=7086506;cid=7086507;cid=7086509;cid=7043324;cid=7085797;cid=7086363;cid=7086379;cid=708639 : : -: 4;cid=7086405;cid=7086505;cid=7086449;cid=7123926;cid=7124057;cid=7123123;cid=7124067;cid=7123939;cid=7124061;cid=7124064;cid=7123937;cid=7124062;cid=7124002; : : -: cid=7124004;cid=7124073;cid=7123987;cid=7123999;cid=7123996;cid=7124025;cid=7124020;cid=7124026;offline_stat=0;online_stat=0;stat_periods=2013-07-12:2013-07-1 : : -: 4,2013-07-01:2013-07-11,2013-07-04:2013-07-11;ulogin=kayak-ru;stat_type=campdate;y1=2013;m1=07;d1=12;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: 0,2013-06-01:2013-06-30;stat_type=campdate;y1=2013;m1=06;d1=01;y2=2013;m2=07;d2=15;group=day;target_all=1&xls=1 : : -: tegoryID=652&search_name=&search_price_from=0&search_price_to=102210¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000¶m_19_23=yep¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: ampStat&target_1=0&reverse=1&sort=clicks&target_all=1&m1=06&online_stat=0&cid=7095075&goals=0&y2=13&offline_stat=0 : : -: ampStat&target_1=0&reverse=1&sort=clicks&target_all=1&online_stat=0&m1=06&cid=7085872&goals=0&y2=13&offline_stat=0 : : -: _type_id=1&product_type_1_model_hyundai[]=Accent&product_type_1_model_hyundai[]=Verna : : -: 0&cmd=showCampStat&reverse=1&sort=clicks&target_all=1&online_stat=0&m1=06&cid=7085610&goals=0&offline_stat=0&y2=13 : : -: tegoryID=648&search_name=ариада&search_price_from=0&search_price_to=2111860¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000& : : -: paramRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 5,2013-06-03:2013-07-14;ulogin=stoli-su;stat_type=campdate;y1=2013;m1=07;d1=08;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -+----------------------------------------------------------------------------------------------------------------------------------------------------------------+------+ -1000 tuples (1.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+----------------------------------------------------------------------------------------------------------------------------------------------------------------+------+ -: : view : -: : s : -+================================================================================================================================================================+======+ -: tegoryID=657&search_name=&search_price_from=0&search_price_to=292381¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_25=0¶mRangeTo_25=10¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 0%26price%3D0;8000%26price_from%3D0%26price_to%3D8000%26category[]%3D224 : : -: mest : : -: tegoryID=710&search_name=&search_price_from=0&search_price_to=799053¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000 : : -: 0,2013-07-01:2013-07-09;stat_type=campdate;y1=2013;m1=07;d1=01;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: tegoryID=657&search_name=&search_price_from=0&search_price_to=292381¶m_27_33=yep¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo : : -: _16=5000¶mRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_25=0¶mRangeTo_25=10¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 01:2013-07-14&m2=07&stat_type=custom&onpage=100&group_by=phrase, retargeting&d1=08&target_1=0&cmd=showCampStat&ulogin=kuncevo-ru&reverse=ARRAY(0x17e9c4c0)&tar : : -: get_all=1&online_stat=0&m1=07&cid=4601559&goals=0&offline_stat=0&y2=13 : : -: ype}&pos={position}&key={keyword} : : -: _type_id=1&product_type_1_model_kia[]=Spectra : : -: duct_type_id=1&product_type_1_model_hyundai[]=Sonata : : -: duct_type_id=1&product_type_1_model_hyundai[]=SantaFe : : -: yundai_comp=find_in_set_any&product_type_id=1&product_type_1_model_hyundai[]=Tucson : : -: %3Dpopularity_desc%26num%3D100%26price_from%3D8000%26price_to%3D11000%26price%3D8000;11000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D100%26price_from%3D24000%26price_to%3D27000%26price%3D24000;27000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D20%26price_from%3D15000%26price_to%3D17000%26price%3D15000;17000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D20%26price_from%3D0%26price_to%3D15000%26price%3D0;15000%26category[]%3D224%26data_form%3DY : : -: 01:2013-07-14&m2=07&stat_type=custom&onpage=10000&group_by=phrase&d1=01&target_1=0&cmd=showCampStat&ulogin=UBRiR66&reverse=1&target_all=1&sort=sum&m1=07&onlin : : -: e_stat=0&cid=6715451&goals=0&y2=13&offline_stat=0 : : -: 1,2013-06-01:2013-07-11;ulogin=pb-context;stat_type=campdate;y1=2013;m1=07;d1=11;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: n=Прочитайте истории успеха клие : : -: 13-06-10:2013-07-14&stat_type=custom&group=week&onpage=10000&group_by=phrase, date&d1=08&target_1=0&cmd=showCampStat&ulogin=izosfera2012&reverse=ARRAY(0x19231 : : -: 210)&target_all=1&online_stat=0&m1=07&cid=5791188&goals=0&offline_stat=0&y2=13 : : -: 013-05-31&m2=07&group=week&d1=15&cmd=showCampStat&reverse=1&detail=yes&target_all=yes&sort=sum&online_stat=1&m1=07&cid=6744826&goals=0&y2=13&offline_stat=0 : : -: 013-04-30&m2=07&group=day&d1=15&cmd=showCampStat&ulogin=yar-tehnostroy&detail=yes&target_all=yes&online_stat=1&m1=06&cid=3539794&goals=0&offline_stat=0&y2=13 : : -: 2,2013-07-08:2013-07-14;stat_type=campdate;y1=2013;m1=06;d1=15;y2=2013;m2=07;d2=15;group=day;target_all=1&xls=1 : : -: sc%26num%3D20%26price%3D0;39000%26price_from%3D0%26price_to%3D39000%26category[]%3D1732 : : -: 02:2013-07-14&m2=07&stat_type=custom&group=day&onpage=100&group_by=date&d1=14&target_1=0&cmd=showCampStat&ulogin=aaamotors2009&reverse=ARRAY(0x4029e50)&target : : -: _all=1&online_stat=1&m1=06&cid=5300235&goals=0&offline_stat=0&y2=13 : : -: tegoryID=658&search_name=&search_price_from=0&search_price_to=845243¶m_27_40=yep¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo : : -: _16=5000¶mRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_22=1¶mRangeTo_22=1500 : : -: ,2013-07-01:2013-07-07&m2=07&group=day&d1=08&cmd=showCampStat&ulogin=conrays-mos&phrasedate=yes&online_stat=0&m1=07&cid=5073455&goals=0&offline_stat=0&y2=13 : : -: 4,2013-07-01:2013-07-15;stat_type=campdate;y1=2013;m1=05;d1=01;y2=2013;m2=05;d2=31;group=month;target_all=1&xls=1 : : -: 10:2013-07-14&m2=07&stat_type=custom&group=week&onpage=10000&group_by=phrase, date&d1=08&cmd=showCampStat&target_1=0&ulogin=avtogermes2013&reverse=1&sort=sum& : : -: target_all=1&online_stat=0&m1=07&cid=5767794&goals=0&y2=13&offline_stat=0 : : -: 7,2013-06-24:2013-06-30;stat_type=campdate;y1=2013;m1=07;d1=08;y2=2013;m2=07;d2=14;group=week;target_all=1&xls=1 : : -: 89779;cid=5688769;cid=5846786;cid=5846794;cid=5846798;cid=5846824;cid=5846814;cid=5846776;cid=5689791;cid=5902210;cid=5843727;cid=5945010;cid=7085900;cid=7086 : : -: 434;cid=7086465;cid=7086475;cid=7086483;cid=7086497;cid=7086504;cid=7086506;cid=7086507;cid=7086509;cid=7043324;cid=7085797;cid=7086363;cid=7086379;cid=708639 : : -: 4;cid=7086405;cid=7086505;cid=7086449;cid=7123926;cid=7124057;cid=7123123;cid=7124067;cid=7123939;cid=7124061;cid=7124064;cid=7123937;cid=7124062;cid=7124002; : : -: cid=7124004;cid=7124073;cid=7123987;cid=7123999;cid=7123996;cid=7124025;cid=7124020;cid=7124026;offline_stat=0;online_stat=0;stat_periods=2013-07-12:2013-07-1 : : -: 4,2013-07-01:2013-07-11,2013-07-04:2013-07-11;ulogin=kayak-ru;stat_type=campdate;y1=2013;m1=07;d1=12;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: 0,2013-06-01:2013-06-30;stat_type=campdate;y1=2013;m1=06;d1=01;y2=2013;m2=07;d2=15;group=day;target_all=1&xls=1 : : -: tegoryID=652&search_name=&search_price_from=0&search_price_to=102210¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000¶m_19_23=yep¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: ampStat&target_1=0&reverse=1&sort=clicks&target_all=1&m1=06&online_stat=0&cid=7095075&goals=0&y2=13&offline_stat=0 : : -: ampStat&target_1=0&reverse=1&sort=clicks&target_all=1&online_stat=0&m1=06&cid=7085872&goals=0&y2=13&offline_stat=0 : : -: _type_id=1&product_type_1_model_hyundai[]=Accent&product_type_1_model_hyundai[]=Verna : : -: 0&cmd=showCampStat&reverse=1&sort=clicks&target_all=1&online_stat=0&m1=06&cid=7085610&goals=0&offline_stat=0&y2=13 : : -: tegoryID=648&search_name=ариада&search_price_from=0&search_price_to=2111860¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000& : : -: paramRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 5,2013-06-03:2013-07-14;ulogin=stoli-su;stat_type=campdate;y1=2013;m1=07;d1=08;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -+----------------------------------------------------------------------------------------------------------------------------------------------------------------+------+ -1000 tuples (1.2s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+----------------------------------------------------------------------------------------------------------------------------------------------------------------+------+ -: : view : -: : s : -+================================================================================================================================================================+======+ -: tegoryID=657&search_name=&search_price_from=0&search_price_to=292381¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_25=0¶mRangeTo_25=10¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 0%26price%3D0;8000%26price_from%3D0%26price_to%3D8000%26category[]%3D224 : : -: mest : : -: tegoryID=710&search_name=&search_price_from=0&search_price_to=799053¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000 : : -: 0,2013-07-01:2013-07-09;stat_type=campdate;y1=2013;m1=07;d1=01;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: tegoryID=657&search_name=&search_price_from=0&search_price_to=292381¶m_27_33=yep¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo : : -: _16=5000¶mRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_25=0¶mRangeTo_25=10¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 01:2013-07-14&m2=07&stat_type=custom&onpage=100&group_by=phrase, retargeting&d1=08&target_1=0&cmd=showCampStat&ulogin=kuncevo-ru&reverse=ARRAY(0x17e9c4c0)&tar : : -: get_all=1&online_stat=0&m1=07&cid=4601559&goals=0&offline_stat=0&y2=13 : : -: ype}&pos={position}&key={keyword} : : -: _type_id=1&product_type_1_model_kia[]=Spectra : : -: duct_type_id=1&product_type_1_model_hyundai[]=Sonata : : -: duct_type_id=1&product_type_1_model_hyundai[]=SantaFe : : -: yundai_comp=find_in_set_any&product_type_id=1&product_type_1_model_hyundai[]=Tucson : : -: %3Dpopularity_desc%26num%3D100%26price_from%3D8000%26price_to%3D11000%26price%3D8000;11000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D100%26price_from%3D24000%26price_to%3D27000%26price%3D24000;27000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D20%26price_from%3D15000%26price_to%3D17000%26price%3D15000;17000%26category[]%3D224%26data_form%3DY : : -: %3Dpopularity_desc%26num%3D20%26price_from%3D0%26price_to%3D15000%26price%3D0;15000%26category[]%3D224%26data_form%3DY : : -: 01:2013-07-14&m2=07&stat_type=custom&onpage=10000&group_by=phrase&d1=01&target_1=0&cmd=showCampStat&ulogin=UBRiR66&reverse=1&target_all=1&sort=sum&m1=07&onlin : : -: e_stat=0&cid=6715451&goals=0&y2=13&offline_stat=0 : : -: 1,2013-06-01:2013-07-11;ulogin=pb-context;stat_type=campdate;y1=2013;m1=07;d1=11;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: n=Прочитайте истории успеха клие : : -: 13-06-10:2013-07-14&stat_type=custom&group=week&onpage=10000&group_by=phrase, date&d1=08&target_1=0&cmd=showCampStat&ulogin=izosfera2012&reverse=ARRAY(0x19231 : : -: 210)&target_all=1&online_stat=0&m1=07&cid=5791188&goals=0&offline_stat=0&y2=13 : : -: 013-05-31&m2=07&group=week&d1=15&cmd=showCampStat&reverse=1&detail=yes&target_all=yes&sort=sum&online_stat=1&m1=07&cid=6744826&goals=0&y2=13&offline_stat=0 : : -: 013-04-30&m2=07&group=day&d1=15&cmd=showCampStat&ulogin=yar-tehnostroy&detail=yes&target_all=yes&online_stat=1&m1=06&cid=3539794&goals=0&offline_stat=0&y2=13 : : -: 2,2013-07-08:2013-07-14;stat_type=campdate;y1=2013;m1=06;d1=15;y2=2013;m2=07;d2=15;group=day;target_all=1&xls=1 : : -: sc%26num%3D20%26price%3D0;39000%26price_from%3D0%26price_to%3D39000%26category[]%3D1732 : : -: 02:2013-07-14&m2=07&stat_type=custom&group=day&onpage=100&group_by=date&d1=14&target_1=0&cmd=showCampStat&ulogin=aaamotors2009&reverse=ARRAY(0x4029e50)&target : : -: _all=1&online_stat=1&m1=06&cid=5300235&goals=0&offline_stat=0&y2=13 : : -: tegoryID=658&search_name=&search_price_from=0&search_price_to=845243¶m_27_40=yep¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo : : -: _16=5000¶mRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_22=1¶mRangeTo_22=1500 : : -: ,2013-07-01:2013-07-07&m2=07&group=day&d1=08&cmd=showCampStat&ulogin=conrays-mos&phrasedate=yes&online_stat=0&m1=07&cid=5073455&goals=0&offline_stat=0&y2=13 : : -: 4,2013-07-01:2013-07-15;stat_type=campdate;y1=2013;m1=05;d1=01;y2=2013;m2=05;d2=31;group=month;target_all=1&xls=1 : : -: 10:2013-07-14&m2=07&stat_type=custom&group=week&onpage=10000&group_by=phrase, date&d1=08&cmd=showCampStat&target_1=0&ulogin=avtogermes2013&reverse=1&sort=sum& : : -: target_all=1&online_stat=0&m1=07&cid=5767794&goals=0&y2=13&offline_stat=0 : : -: 7,2013-06-24:2013-06-30;stat_type=campdate;y1=2013;m1=07;d1=08;y2=2013;m2=07;d2=14;group=week;target_all=1&xls=1 : : -: 89779;cid=5688769;cid=5846786;cid=5846794;cid=5846798;cid=5846824;cid=5846814;cid=5846776;cid=5689791;cid=5902210;cid=5843727;cid=5945010;cid=7085900;cid=7086 : : -: 434;cid=7086465;cid=7086475;cid=7086483;cid=7086497;cid=7086504;cid=7086506;cid=7086507;cid=7086509;cid=7043324;cid=7085797;cid=7086363;cid=7086379;cid=708639 : : -: 4;cid=7086405;cid=7086505;cid=7086449;cid=7123926;cid=7124057;cid=7123123;cid=7124067;cid=7123939;cid=7124061;cid=7124064;cid=7123937;cid=7124062;cid=7124002; : : -: cid=7124004;cid=7124073;cid=7123987;cid=7123999;cid=7123996;cid=7124025;cid=7124020;cid=7124026;offline_stat=0;online_stat=0;stat_periods=2013-07-12:2013-07-1 : : -: 4,2013-07-01:2013-07-11,2013-07-04:2013-07-11;ulogin=kayak-ru;stat_type=campdate;y1=2013;m1=07;d1=12;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -: 0,2013-06-01:2013-06-30;stat_type=campdate;y1=2013;m1=06;d1=01;y2=2013;m2=07;d2=15;group=day;target_all=1&xls=1 : : -: tegoryID=652&search_name=&search_price_from=0&search_price_to=102210¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000¶mRa : : -: ngeFrom_17=1¶mRangeTo_17=5000¶m_19_23=yep¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: ampStat&target_1=0&reverse=1&sort=clicks&target_all=1&m1=06&online_stat=0&cid=7095075&goals=0&y2=13&offline_stat=0 : : -: ampStat&target_1=0&reverse=1&sort=clicks&target_all=1&online_stat=0&m1=06&cid=7085872&goals=0&y2=13&offline_stat=0 : : -: _type_id=1&product_type_1_model_hyundai[]=Accent&product_type_1_model_hyundai[]=Verna : : -: 0&cmd=showCampStat&reverse=1&sort=clicks&target_all=1&online_stat=0&m1=06&cid=7085610&goals=0&offline_stat=0&y2=13 : : -: tegoryID=648&search_name=ариада&search_price_from=0&search_price_to=2111860¶mRangeFrom_18=1¶mRangeTo_18=5000¶mRangeFrom_16=1¶mRangeTo_16=5000& : : -: paramRangeFrom_17=1¶mRangeTo_17=5000¶mRangeFrom_21=1¶mRangeTo_21=1200 : : -: 5,2013-06-03:2013-07-14;ulogin=stoli-su;stat_type=campdate;y1=2013;m1=07;d1=08;y2=2013;m2=07;d2=14;group=day;target_all=1&xls=1 : : -+----------------------------------------------------------------------------------------------------------------------------------------------------------------+------+ -1000 tuples (1.3s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageVi ews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -: icso : chen : ngin : : : iews : -: urce : gine : eid : : : : -: id : id : : : : : -+======+======+======+====================================================================+=====================================================================+=======+ -: : : : : ype=campdate : : -: : : : type=campdate : : : -: : : : : &payment_type=1 : : -: : : : : d=7 : : -: : : : : &mediaType=text : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0 : : : -: : : : : 79.0&authredirlevel=1373600157.0&authredirlevel=1373081665.0&authre : : -: : : : : dirlevel=1372822331.0&authredirlevel=1372563095.0&authredirlevel=13 : : -: : : : : 72303801.0&authredirlevel=1372044532.0&authredirlevel=1371730939.0& : : -: : : : : authredirlevel=1371471570.0&authredirlevel=1371212358.0&cmd=showCam : : -: : : : : p&cid=2856748 : : -: : : : t : : : -: : : : : ype=campdate : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : -: : : : : s=1 : : -: : : : : owCamps : : -: : : : 1&mediaType=text : : : -: : : : : dirlevel=1373731047.0&cmd=showCamps : : -: : : : : 09.0&authredirlevel=1370955451.0&cmd=showCamp&cid=5477450 : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : type=campdate : : : -: : : : : d=7 : : -: : : : : 47.0&cmd=showCamp&cid=4567586 : : -: : : : : 6 : : -: : : : : OqWshMF.pl?cmd=showCamps&emedia_camps=0&ulogin=lena-san-std : : -: : : : : n= : : -: : : : : 82.0&cmd=showCamp&cid=2753896 : : -: : : : : 5 : : -: : : : 75 : : : -: : : : : &mediaType=text : : -: : : : : 0 : : -: : : : : 2 : : -: : : : 19 : : : -: : : : 93 : : : -: : : : : 9 : : -: : : : : dirlevel=1373869482.0&cmd=showCamp&cid=2997085 : : -: : : : : 3 : : -: : : : 21 : : : -: : : : : 88.0 : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 01.0&cmd=showCamp&cid=4567586 : : -: : : : howCampStat&cid=7056273&offline_stat=0&online_stat=1&stat_periods= : 56273&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y2= : : -: : : : 2013-05-25:2013-06-24,2013-05-22:2013-05-23,2013-05-17:2013-05-18& : 13&group=day : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : : -: : : : &goals=0&target_all=yes : : : -: : : : ks=1 : : : -: : : : : 1 : : -: : : : : 1 : : -: : : : : 7 : : -: : : : =total&target_all=1&page=1&cmd=showCampStat&cid=3016175&show_banne : owCampStat&cid=3016175&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : rs_stat=1 : 12-02-03:2012-02-04,2011-08-01:2011-09-04,2010-12-29:2011-01-02&typ : : -: : : : : es=total&page=&group=day&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&sh : : -: : : : : ow_banners_stat=1 : : -: : : : 37 : : : -: : : : 65 : : : -: : : : 01 : : : -: : : : : 5 : : -: : : : : 9 : : -: : : : 19 : : : -: : : : 71 : : : -: : : : : 1 : : -: : : : 08 : : : -: : : : 81 : : : -: : : : : 1 : : -: : : : 45 : : : -: : : : : 0 : : -: : : : : 5 : : -: : : : : 8 : : -: : : : type=campdate : : : -: : : : 90 : gn_direct_plastik_den : : -: : : : : 3 : : -: : : : 40 : : : -: : : : 39 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 03 : : : -: : : : : 6 : : -: : : : ks=1 : : : -: : : : : 7 : : -: : : : : 2 : : -: : : : howCamp&cid=5940082 : : : -: : : : : 9 : : -: : : : 7-.31620-2.1373861604902917.198343986.198343986..64036780.1406.9b1 : 7 : : -: : : : : 1 : : -: : : : : Informers.html : : -: : : : d : : : -: : : : 29 : : : -: : : : 95 : : : -: : : : ransfer : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6870158 : : -: : : : 68 : : : -: : : : : dirlevel=1342185655.0&cmd=showCamps&ulogin=beautyland-artha8 : : -: : : : : 1 : : -: : : : : 8 : : -: : : : 71 : : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 4 : : -: : : : : 018058&cmd=showCamp : : -: : : : 76 : owCampStat&cid=5109476&detail=Yes&types=days : : -: : : : : 9 : : -: : : : : owSubClientCamps : : -: : : : : 1 : : -: : : : : 0 : : -: : : : type=campdate : : : -: : : : 91 : : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=4718384 : : -: : : : : w.advertising.yandex.ru/welcome/pdf/direct_booklet.pdf : : -: : : : : =Yes&types=days&cid=7114664 : : -: : : : : 7 : : -: : : : : &mediaType=text : : -: : : : : 51.0 : : -: : : : ditBannerEasy&cid=7080496&bid=188854799 : 080496&cmd=showCamp : : -: : : : : 8 : : -: : : : 56 : owCampStat&cid=5895656&detail=Yes&types=days : : -: : : : edirlevel=1373786750.0&cmd=showCampStat&cid=6421801&cid=6424294&ci : : : -: : : : d=6487283&cid=6493829&cid=6494462&cid=6495189&stat_type=campdate&c : : : -: : : : srf_token=CzqM8_OaIwxa28-a : : : -: : : : 28 : : : -: : : : : 6 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : type=campdate : : : -: : : : 77 : lshoj-kvartiry : : -: : : : 83 : : : -: : : : : vancedForecast : : -: : : : howCamp&cid=2925555 : owCamp&cid=2925555 : : -: : : : howCamps&ulogin=aplarru : 9&ulogin=aplarru : : -: : : : : 1 : : -: : : : 16 : : : -: : : : : ger : : -: : : : : 52.0&authredirlevel=1373884890.0 : : -: : : : l=Yes&types=days&cid=7026863 : : : -: : : : : 4 : : -: : : : : 5 : : -: : : : 43 : : : -: : : : : 1 : : -: : : : : 9 : : -: : : : : owCamp&cid=7129447#193577203 : : -: : : : 38 : 46.0 : : -: : : : : 8 : : -: : : : 44 : : : -: : : : : 8 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6486629 : : -: : : : : 65.0 : : -: : : : : =Yes&types=days&cid=7131552 : : -: : : : : 3 : : -: : : : howCamp&cid=7129026&tab=wait : owCamp&cid=7129026&tab=active : : -: : : : : 3 : : -: : : : : =Yes&types=days&cid=7015594 : : -: : : : 48 : : : -: : : : : 5 : : -: : : : : 3 : : -: : : : 78 : : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=5826713 : : : -: : : : : 5 : : -: : : : : 1 : : -: : : : : =Yes&types=days&cid=5309194 : : -: : : : howCamps : owCamps : : -: : : : : 6 : : -: : : : 76 : : : -: : : : 21 : : : -: : : : 55 : : : -: : : : : 5 : : -: : : : : 1 : : -: : : : : 8 : : -: : : : : 6 : : -: : : : : 6 : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : : -: : : : : type=campdate : : -: : : : t : : : -: : : : l=Yes&types=days&cid=6752443 : 52443&phrasedate=Yes&target_all=1 : : -: : : : howCamp&cid=6501536&tab=&page=2 : : : -: : : : : 8 : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=7026863 : : -: : : : 41 : : : -: : : : : 5 : : -: : : : : owCamps&ulogin=super-svictor : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6411871 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6189242 : : -: : : : l=Yes&cid=6009684&showCampStat=cmd&types=days&target_all=1&d1=08&m : owCampStat&cid=6009684&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : 1=07&y1=13&d2=08&m2=07&y2=13&group=day : 13-02-25:2013-02-25,2013-02-26:2013-02-26,&detail=yes&page=&y1=2013 : : -: : : : : &m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : 1&mediaType=text : : : -: : : : : =Yes&types=days&cid=6752443 : : -: : : : 42 : : : -: : : : : 3 : : -: : : : howCamps : 2 : : -: : : : 263.0&cmd=showCamp&cid=5830159 : owCampStat&cid=5830159&detail=Yes&types=days : : -: : : : : 7 : : -: : : : : =Yes&types=days&cid=6467205 : : -: : : : : =Yes&types=days&cid=6467203 : : -: : : : 89 : : : -: : : : howCamp&cid=5088603 : owCamp&cid=5088603 : : -: : : : 77 : : : -: : : : : 6 : : -: : : : : 78.0 : : -: : : : 42 : : : -: : : : 38 : owCampStat&cid=6356238&detail=Yes&types=days : : -: : : : howCamps : 6 : : -: : : : : 9 : : -: : : : howCamps&ulogin=discoverycruise : =Yes&types=days&cid=7065506&ulogin=discoverycruise : : -: : : : : 33.0&cmd=showCampStat&detail=Yes&types=days&cid=6393197 : : -: : : : : 7 : : -: : : : ditCamp&cid=5309194 : =Yes;types=days;cid=5309194 : : -: : : : : owCamp&cid=6980076 : : -: : : : : dirlevel=1373863808.0&authredirlevel=1373522397.0&cmd=showCamp&cid= : : -: : : : : 6051926&tab=active : : -: : : : : =easy&from=metrika : : -: : : : : =Yes&types=days&cid=7135716 : : -: : : : : 3 : : -: : : : howCamp&cid=7129447&tab=wait : owCamp&cid=7129447&tab=active : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : 5279956&tab=media : owCamp&cid=5279956&tab=active : : -: : : : ditBannerEasy&cid=6835814&bid=171781249 : 835814&cmd=showCamp : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6931378 : : -: : : : : ype=campdate : : -: : : : 68 : : : -: : : : 42 : : : -: : : : l=Yes&types=days&cid=4718384 : : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 7 : : -: : : : : owCamps&ulogin=visacity08 : : -: : : : : 109741&cmd=showCamp : : -: : : : type=campdate&&ulogin=vnukovo-piter : owCamps&ulogin=vnukovo-piter : : -: : : : : 1 : : -: : : : 3243479&tab=media : owCamp&cid=3243479&tab=active : : -: : : : : 1 : : -: : : : ditBannerEasy&cid=7123828&bid=193277936 : 123828&cmd=showCamp : : -: : : : : 6 : : -: : : : : 0 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : howCamp&cid=7133264 : owCamp&cid=7133264&tab=wait : : -: : : : : 5 : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : howCamps&ulogin=well-mir-ru : 0&ulogin=well-mir-ru : : -: : : : 66 : : : -: : : : 83 : : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : 5 : : -: : : : : 4 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : : owCamps&ulogin=modul-sdskbeton : : -: : : : : =Yes&types=days&cid=7064245 : : -: : : : &ulogin= : : : -: : : : : 3 : : -: : : : howCampStat&cid=3370690&detail=Yes&types=days : 0 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : : 3 : : -: : : : : owCamps : : -: : : : : 7 : : -: : : : : come=yes : : -: : : : : 3 : : -: : : : : 9 : : -: : : : : 3 : : -: : : : : 6 : : -: : : : howCampStat&cid=7109552&detail=Yes&types=days : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7109552&sh : : -: : : : : ow_banners_stat=1 : : -: : : : howCamp&cid=7134723 : : : -: : : : howCamps&ulogin=alfenjoy-dir : 3&ulogin=alfenjoy-dir : : -: : : : howCamps&ulogin=allstarshop-yd : owCamps&ulogin=allstarshop-yd : : -: : : : howCamps : 7 : : -: : : : : 6 : : -: : : : 35 : : : -: : : : 96&ulogin=kuncevo-ru : vo-avto-obyavlyaet-start-prodazh-LEXUS-ES-i-LEXUS-RX-spetsialnyh-ve : : -: : : : : rsiy.tmex : : -: : : : : =Yes&types=days&cid=6144446 : : -: : : : : 2 : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 137413&cmd=orderCamp&agree=yes : : : -: : : : : 8 : : -: : : : 80 : : : -: : : : 994219&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y : 9 : : -: : : : 2=13&group=day : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=3346173 : : -: : : : : =Yes&types=days&cid=4300226 : : -: : : : 90 : : : -: : : : id=7 : : : -: : : : : owCampStat&cid=4194803&cid=5256554&cid=5718709&cid=6790267&cid=6790 : : -: : : : : 271&cid=7070376&stat_type=campdate&ulogin=mybalcony-ru&csrf_token=L : : -: : : : : BcSJLrXLKh9zXYq : : -: : : : 38 : owCampStat&cid=6356238&detail=Yes&types=days : : -: : : : 431468&phrasedate=Yes&target_all=1 : owCampStat&cid=5431468&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-06-27:2013-07-10,2013-01-01:2013-06-01,2013-06-01:2013-07-03&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 3 : : -: : : : 74 : : : -: : : : all : : : -: : : : : 7 : : -: : : : : 0 : : -: : : : 16 : : : -: : : : : owClients : : -: : : : : 6 : : -: : : : : =Yes&types=days&cid=1475303 : : -: : : : : 2 : : -: : : : l=Yes&types=days&cid=6942653 : owCampStat&cid=6942653&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-06-22:2013-06-22,2013-06-16:2013-06-18,2013-06-17:2013-06-18&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : 97 : : : -: : : : howCamps : 6 : : -: : : : 51 : : : -: : : : : 480202&cmd=showCamp : : -: : : : howCamp&cid=3973331&ulogin=kuncevo-ru : : : -: : : : : owCamps&ulogin=visamaster08 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=6800439 : : : -: : : : 09 : owCamp&cid=6797009 : : -: : : : : 9 : : -: : : : : dirlevel=1373859813.0&cmd=showCampStat&cid=5538008&detail=Yes&types : : -: : : : : =days : : -: : : : 316.0&cmd=showCamp&cid=5993938 : : : -: : : : : 2 : : -: : : : : 7 : : -: : : : : =Yes&types=days&cid=7103125 : : -: : : : : 8 : : -: : : : howCamp&cid=7129447&tab=wait : owCamp&cid=7129447&tab=active : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6883763 : : -: : : : : 7 : : -: : : : : ype=campdate : : -: : : : : =Yes&types=days&cid=1724608 : : -: : : : : 5 : : -: : : : : 2 : : -: : : : : 3 : : -: : : : : 2 : : -: : : : 09 : : : -: : : : : 4 : : -: : : : : ed : : -: : : : = : 17.0&authredirlevel=1373534053.0&cmd=apiApplicationList&ulogin= : : -: : : : : 2 : : -: : : : HB.31620-2.1373870178024287.199046816.199046816..14150262.1406.c65 : : : -: : : : &rnd=0.9338765616586568 : : : -: : : : 368.0&cmd=showCamp&cid=4685945 : 74.0&authredirlevel=1363929368.0&cmd=showCamp&cid=4685945 : : -: : : : : 5 : : -: : : : : 97.0&cmd=ForecastByWords : : -: : : : 71 : : : -: : : : howCampStat&cid=2196876&detail=Yes&types=days : 6 : : -: : : : = : : : -: : : : howCampStat&cid=7056273&offline_stat=0&online_stat=1&stat_periods= : 56273&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y2= : : -: : : : 2013-05-25:2013-06-24,2013-05-22:2013-05-23,2013-05-17:2013-05-18& : 13&group=day : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : : -: : : : &goals=0&target_all=yes : : : -: : : : : 9 : : -: : : : edirlevel=1373868161.0&&cid=7027214&cmd=showCamp : owCampStat&cid=7027214&detail=Yes&types=days : : -: : : : : =Yes&types=days&cid=6630335 : : -: : : : : owCamps : : -: : : : nt_country=225&cmd=editBannerEasy¤cy=YND_FIXED&interface_typ : 136912&cmd=showCamp&unglued=194242390 : : -: : : : e=easy¬new=yes : : : -: : : : : 7 : : -: : : : : 6 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-11:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-03:201 : : -: : : : 013-07-15,2013-07-01:2013-07-14,2013-07-05:2013-07-13&stat_type=ca : 3-07-15,2013-07-11:2013-07-15,2013-07-01:2013-07-14&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&target_al : ate&y1=2013&m1=07&d1=03&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : l=1 : : : -: : : : : 83.0 : : -: : : : howCamps&ulogin=nordex-grandit : 57.0&cmd=showCampStat&detail=Yes&types=days&cid=6947313&ulogin=nord : : -: : : : : ex-grandit : : -: : : : : =Yes&types=days&cid=7068093 : : -: : : : : =Yes&types=days&cid=6834748 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2011-10-27:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2011-10-27:201 : : -: : : : 011-10-27,2011-08-31:2011-08-31,2011-08-26:2011-08-26&stat_type=ca : 1-10-27,2011-08-31:2011-08-31,2011-08-26:2011-08-26&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=14&y2=2013&m2=07&d2=14&group=day&target_al : ate&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : l=1 : : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 38 : 07.0&authredirlevel=1373876446.0 : : -: : : : : 134782&cmd=showCamp&fixated=150055634 : : -: : : : : dirlevel=1364366494.0&cmd=showCamp&cid=6106774 : : -: : : : : =Yes&types=days&cid=7072622 : : -: : : : : owClients : : -: : : : : 113103&cmd=showCamp : : -: : : : : 2 : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : : 5 : : -: : : : : 05.6&authredirlevel=1372014905.5&authredirlevel=1372014905.4&authre : : -: : : : : dirlevel=1372014905.3&authredirlevel=1372014905.2&authredirlevel=13 : : -: : : : : 72014905.1&authredirlevel=1372014905.0&authredirlevel=1370480737.6& : : -: : : : : authredirlevel=1370480737.5&authredirlevel=1370480737.4&authredirle : : -: : : : : vel=1370480737.3&authredirlevel=1370480737.2&authredirlevel=1370480 : : -: : : : : 737.1&authredirlevel=1370480737.0&cmd=showCampStat&stat_type=campda : : -: : : : : te&target_0=1&target_1=1&target_all=1 : : -: : : : : ulogin= : : -: : : : : 5 : : -: : : : howCamp&cid=6980076 : owCampStat&cid=6980076&detail=Yes&types=days : : -: : : : &ulogin= : ulogin= : : -: : : : howClients : owCamps&ulogin=K-PV-beeline : : -: : : : l=Yes&types=days&cid=88689 : : : -: : : : 32 : : : -: : : : 15 : owCampStat&cid=5991615&detail=Yes&types=days : : -: : : : howCamp&cid=6980076 : owCamp&cid=6980076 : : -: : : : : =Yes&types=days&cid=5871623 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 8 : : -: : : : : owCampStat&cid=4917291&detail=Yes&types=days : : -: : : : : 2 : : -: : : : : 90.0 : : -: : : : : 8 : : -: : : : : owCamps&ulogin=domusmebel-ru : : -: : : : : 0 : : -: : : : : =Yes&types=days&cid=4735372 : : -: : : : : 9 : : -: : : : 49 : : : -: : : : : 5 : : -: : : : 08 : : : -: : : : howCampStat&cid=3353427&offline_stat=0&online_stat=0&stat_periods= : d2=15&target_0=0&stat_periods=2013-07-01:2013-07-06,2013-06-27:2013 : : -: : : : 2013-07-01:2013-07-06,2013-06-27:2013-07-05,2013-06-01:2013-06-30& : -07-05,2013-06-01:2013-06-30&m2=07&stat_type=custom&onpage=100&grou : : -: : : : stat_type=custom&sort=&reverse=ARRAY(0x1639e200)&group_by=tag&filt : p_by=tag, banner, page, phrase&d1=15&target_1=0&cmd=showCampStat&ta : : -: : : : er_tag=&group_by=banner&filter_banner=&group_by=page&filter_page=& : rget_all=1&online_stat=0&m1=07&cid=3353427&goals=0&offline_stat=0&y : : -: : : : filter_page_target=&group_by=phrase&filter_phrase=&goals=0&y1=2013 : 2=13&sort=clicks&reverse=1 : : -: : : : &m1=07&d1=15&y2=2013&m2=07&d2=15&onpage=100 : : : -: : : : : owCamps : : -: : : : : 2 : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 8 : : -: : : : 44 : owCampStat&cid=4154744&detail=Yes&types=days : : -: : : : 01 : : : -: : : : site_promotion_search : : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 02.0 : : -: : : : : 7 : : -: : : : howCamp&cid=6522990&tab=off : owCamp&cid=6522990 : : -: : : : : 9 : : -: : : : : 8 : : -: : : : : 9 : : -: : : : : 5 : : -: : : : : 4 : : -: : : : 813.0&cmd=showCamp&cid=5210661 : : : -: : : : : 63.0&cmd=showCamp&cid=5830159 : : -: : : : : =Yes&types=days&cid=6993362 : : -: : : : : vancedForecast : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : 7090667&cmd=showCamp : owCampStat&cid=7090667&detail=Yes&types=days : : -: : : : 77&ulogin=svk-direct : : : -: : : : : 3 : : -: : : : : 1 : : -: : : : howCampStat&cid=5975585&detail=Yes&types=days : 5 : : -: : : : : 8 : : -: : : : : 1 : : -: : : : : 41.0&authredirlevel=1373879619.0&cmd=showCamp&cid=5356346 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-03-01:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-03-01:201 : : -: : : : 013-07-15,2013-06-01:2013-06-14,2013-06-01:2013-07-15&stat_type=ca : 3-07-15,2013-06-01:2013-06-14,2013-06-01:2013-07-15&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=month&target_ : ate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : all=1 : : : -: : : : : owCamps&ulogin=infoprom51 : : -: : : : : 64.0&cmd=showCamp&cid=5028648 : : -: : : : 64 : : : -: : : : : 6 : : -: : : : : 34.0&cmd=ForecastByWords : : -: : : : : 4 : : -: : : : ts : : : -: : : : : 2 : : -: : : : ncy : : : -: : : : : 8 : : -: : : : : 55.0&cmd=showCampStat&detail=Yes&types=days&cid=6849113 : : -: : : : howCamps&ulogin=aplarru : 9&ulogin=aplarru : : -: : : : : =Yes&types=days&cid=5039376 : : -: : : : : owCamp&cid=4748947&ulogin=cont5nn#71488518 : : -: : : : _camps=0&cmd=showSubClientCamps&authredirlevel=1373873308.0&settin : dirlevel=1373880608.0&media_camps=0&cmd=showSubClientCamps&authredi : : -: : : : gs_sorted=&sort=total&reverse=1 : rlevel=1373873308.0&settings_sorted=&sort=total&reverse=0 : : -: : : : howCamp&cid=4581573&tab=decline : owCamp&cid=4581573&tab=active : : -: : : : : 9 : : -: : : : howClients : vancedForecast : : -: : : : : 983763&cmd=showCamp&unglued=194305392 : : -: : : : : 9 : : -: : : : : 5 : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=7030009 : : -: : : : 0 : : : -: : : : : 8 : : -: : : : : 9 : : -: : : : : 65.0&cmd=showCamp&cid=7135509 : : -: : : : 642.0 : ype=campdate : : -: : : : : 5 : : -: : : : 35 : 5 : : -: : : : : dirlevel=1373877895.0&cmd=showCamp&cid=3055906 : : -: : : : : 6 : : -: : : : 099442&cid=7100140&cid=7100144&cid=7100148&cid=7100150&cid=7132172 : owCampStat&cid=7099442&cid=7100140&cid=7100144&cid=7100148&cid=7100 : : -: : : : &cid=7132174&cid=7132175&cid=7132177&cid=7132179&cid=7132632&cid=7 : 150&cid=7132172&cid=7132174&cid=7132175&cid=7132177&cid=7132179&cid : : -: : : : 132838&cid=7132888&cid=7133059&cid=7133217&stat_type=campdate&csrf : =7132632&cid=7132838&cid=7132888&cid=7133059&cid=7133217&offline_st : : -: : : : _token=aENmaxzslTI5oMqD : at=0&online_stat=0&stat_periods=2013-07-01:2013-07-13,2013-07-01:20 : : -: : : : : 13-07-12,2013-06-14:2013-07-11&stat_type=campdate&y1=2013&m1=07&d1= : : -: : : : : 15&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : 35 : 63.0&authredirlevel=1373873632.0 : : -: : : : : 1 : : -: : : : 73 : : : -: : : : howCampStat&cid=6981382&offline_stat=0&online_stat=1&stat_periods= : owCampStat&cid=6981382&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : 2013-06-10:2013-06-25,2013-06-10:2013-06-24,&detail=yes&page=&y1=2 : 13-06-10:2013-06-25,2013-06-10:2013-06-24,&detail=yes&page=&y1=2013 : : -: : : : 013&m1=07&d1=05&y2=2013&m2=07&d2=12&group=day&goals=0&target_all=y : &m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : es : : : -: : : : : =Yes&types=days&cid=6621650 : : -: : : : gSearch&who=camps&cid=&camp_login=&camp_name=атлант : 4&ulogin=atlant-b2bnew : : -: : : : : 0 : : -: : : : : owCamps&ulogin=domusmebel-ru : : -: : : : : 3 : : -: : : : : owCamp&cid=3601040 : : -: : : : : =Yes&types=days&cid=6858643 : : -: : : : : =Yes&types=days&cid=7109462 : : -: : : : : =Yes&types=days&cid=7085888 : : -: : : : 25 : : : -: : : : : owCamps : : -: : : : d=18770681&resp_desc=success : owCamps : : -: : : : howCamp&cid=6980076 : owCampStat&cid=6980076&detail=Yes&types=days : : -: : : : : 5 : : -: : : : : 4 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=5129796 : : -: : : : : 7 : : -: : : : howCampStat&cid=7104084&detail=Yes&types=days : 104084&cmd=showCamp : : -: : : : : 6 : : -: : : : : 2 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=3427083 : : : -: : : : : =Yes&types=days&cid=7060114 : : -: : : : l=Yes&types=days&cid=6833696 : owCampStat&cid=6833696&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-20:2013-06-21,&detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m : : -: : : : : 2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : : 137972&cmd=showCamp&unglued=194336987 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : &mediaType=text&firstbanner=std : : -: : : : 137830&cmd=orderCamp&agree=yes : : : -: : : : : 9 : : -: : : : 09 : : : -: : : : : 131075&cmd=showCamp : : -: : : : : 2 : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6960697 : : -: : : : 7129022&cmd=showCamp : owCampStat&cid=7129022&detail=Yes&types=days : : -: : : : 43 : : : -: : : : : 8 : : -: : : : 17 : owCamp&cid=7082517 : : -: : : : : 6 : : -: : : : 31&ulogin=kuncevo-ru : : : -: : : : 394451&phrasedate=Yes&target_all=1 : owCampStat&cid=4394451&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 12-01-01:2012-11-21,2012-01-01:2012-07-31,2012-03-23:2012-03-23&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 1 : : -: : : : howCamps : owCamps : : -: : : : : 1 : : -: : : : : 6 : : -: : : : gSearch&who=camps&cid=&camp_login=&camp_name=дизв : 9&ulogin=dizvitr-b2b : : -: : : : : 7 : : -: : : : : 9 : : -: : : : : campdate : : -: : : : : 0 : : -: : : : : 6 : : -: : : : : 6 : : -: : : : howCamps&ulogin=oknabuduschego : =Yes&types=days&cid=5506368&ulogin=oknabuduschego : : -: : : : howClients&sort=total&reverse=0 : owCamps&ulogin=Rusokon2007 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=4879898 : : -: : : : 1&mediaType=text : : : -: : : : 85 : : : -: : : : howCamps&ulogin=visacity08 : 5&ulogin=visacity08 : : -: : : : 56 : : : -: : : : : owCamps&ulogin=dc12a : : -: : : : 59 : : : -: : : : howCamps&ulogin=ul-smilestar-su : =ul-smilestar-su#list : : -: : : : howCampStat&cid=7053221&detail=Yes&types=days : owCampStat&cid=7053221&offline_stat=0&online_stat=0&stat_periods=&d : : -: : : : : etail=yes&page=&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&g : : -: : : : : oals=0&target_all=yes&show_banners_stat=1 : : -: : : : : =Yes&types=days&cid=7133318 : : -: : : : : 5 : : -: : : : 25 : : : -: : : : howCamp&tab=all&cid=7126549&ulogin=transkontinental2014#194270034 : : : -: : : : : 2 : : -: : : : 02 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 07 : : : -: : : : : 1 : : -: : : : : 9 : : -: : : : howCamp&cid=7078898&tab=off : owCamp&cid=7078898 : : -: : : : 01 : : : -: : : : : 4 : : -: : : : : 35.0 : : -: : : : : owCampStat&cid=2886087&detail=Yes&types=days : : -: : : : : 8 : : -: : : : type=campdate : dirlevel=1373865822.0&cmd=showCampStat&offline_stat=0&online_stat=0 : : -: : : : : &stat_periods=2013-07-04:2013-07-05,2013-06-10:2013-06-13,2013-05-2 : : -: : : : : 1:2013-05-31&stat_type=campdate&y1=2013&m1=07&d1=15&y2=2013&m2=07&d : : -: : : : : 2=15&group=day&target_all=1 : : -: : : : : 0 : : -: : : : : owCamp&cid=6354573#194189392 : : -: : : : : owCamps : : -: : : : 25 : : : -: : : : : 8 : : -: : : : : 04.0&cmd=showCamp&cid=7129428 : : -: : : : howCamp&cid=7130323 : ids=193775525&cid=7130323 : : -: : : : howCamps&ulogin=richton-studio : 7&ulogin=richton-studio : : -: : : : : owCampStat&cid=7034406&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-07-11:2013-07-11,2013-07-12:2013-07-12,2013-07-13:2013-07-13&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 0 : : -: : : : howCamp&tab=decline&cid=6886514 : owCamp&cid=6886514&tab=active : : -: : : : : 33.0 : : -: : : : : 60.0 : : -: : : : : 5 : : -: : : : : dirlevel=1373866480.0&cmd=showCampStat&stat_type=by_agency_clients : : -: : : : : 9 : : -: : : : : 463697&cmd=showCamp : : -: : : : 28 : : : -: : : : : 6 : : -: : : : : 8 : : -: : : : : 4 : : -: : : : 60 : : : -: : : : : 5 : : -: : : : : 7 : : -: : : : : 1 : : -: : : : howCampStat&cid=6456112&offline_stat=0&online_stat=0&stat_periods= : owCampStat&cid=6456112&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : 2013-07-10:2013-07-10,2013-07-09:2013-07-09,2013-07-08:2013-07-08& : 13-07-10:2013-07-10,2013-07-09:2013-07-09,2013-07-08:2013-07-08&phr : : -: : : : phrasedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : =day&goals=0&target_all=yes : &goals=0&target_all=yes : : -: : : : : 7 : : -: : : : : 3 : : -: : : : howCampStat&cid=4121349&detail=Yes&types=days : owCampStat&cid=4121349&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-07-08:2013-07-09,2013-06-01:2013-06-06,2013-04-01:2013-06-03&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : : owClients : : -: : : : 15 : : : -: : : : l=Yes&types=days&cid=7103094 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7103094&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 4 : : -: : : : 84 : : : -: : : : 89 : : : -: : : : 1&mediaType=text : : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 2 : : -: : : : 86 : : : -: : : : : 4 : : -: : : : 12 : : : -: : : : : 6 : : -: : : : : owCamps&ulogin=dc12a : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 9 : : -: : : : l=Yes&types=days&cid=141133 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=141133&sho : : -: : : : : w_banners_stat=1 : : -: : : : : 108927&cmd=showCamp : : -: : : : howCamp&cid=6886521&tab=off : owCamp&cid=6886521 : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=7133822 : : -: : : : : owCamps&ulogin=mitsubishi-galant2013 : : -: : : : ditCamp&cid=7088102 : owCamp&cid=7088102 : : -: : : : l=Yes&types=days&cid=5033617 : : : -: : : : : 2 : : -: : : : 56 : : : -: : : : : =Yes&types=days&cid=5056684 : : -: : : : 25 : : : -: : : : : owCamps : : -: : : : l=Yes&types=days&cid=6479409 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=6479409&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 9 : : -: : : : : 85.0 : : -: : : : : =Yes&types=days&cid=7133333 : : -: : : : : owCampStat&cid=3618668&detail=Yes&types=days : : -: : : : 74 : owCamp&cid=4275274&tag=303211 : : -: : : : : =Yes&types=days&cid=3298907 : : -: : : : : 9 : : -: : : : : 1 : : -: : : : howCamp&cid=5931713&tab=wait : owCamp&cid=5931713&tab=active : : -: : : : : &mediaType=text : : -: : : : 22 : : : -: : : : 002.0 : 7 : : -: : : : : e_stat=0&online_stat=0&stat_type=campdate&group=day&target_0=1&targ : : -: : : : : et_1=1 : : -: : : : : 4 : : -: : : : mportCampSuccess&cid=4202002&make_new_camp=0 : owCamp&cid=4202002 : : -: : : : : 5 : : -: : : : 87 : : : -: : : : howCamps : owCamps : : -: : : : 99 : : : -: : : : : 3 : : -: : : : 90 : 0 : : -: : : : edirlevel=1373611946.0&authredirlevel=1372140313.0&authredirlevel= : : : -: : : : 1371447906.0&authredirlevel=1368426548.0&authredirlevel=1367818443 : : : -: : : : .0&cmd=showCamps&ulogin=Newdesign-Larus : : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : 97 : : : -: : : : : 9 : : -: : : : howCamps&ulogin=ogurcova-mkt : =Yes&types=days&cid=4139609&ulogin=ogurcova-mkt : : -: : : : : 2 : : -: : : : : 2 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : : 136655&cmd=showCamp : : -: : : : : 8 : : -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -1000 tuples (7.2s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageVi ews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -: icso : chen : ngin : : : iews : -: urce : gine : eid : : : : -: id : id : : : : : -+======+======+======+====================================================================+=====================================================================+=======+ -: : : : : ype=campdate : : -: : : : type=campdate : : : -: : : : : &payment_type=1 : : -: : : : : d=7 : : -: : : : : &mediaType=text : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0 : : : -: : : : : 79.0&authredirlevel=1373600157.0&authredirlevel=1373081665.0&authre : : -: : : : : dirlevel=1372822331.0&authredirlevel=1372563095.0&authredirlevel=13 : : -: : : : : 72303801.0&authredirlevel=1372044532.0&authredirlevel=1371730939.0& : : -: : : : : authredirlevel=1371471570.0&authredirlevel=1371212358.0&cmd=showCam : : -: : : : : p&cid=2856748 : : -: : : : t : : : -: : : : : ype=campdate : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : -: : : : : s=1 : : -: : : : : owCamps : : -: : : : 1&mediaType=text : : : -: : : : : dirlevel=1373731047.0&cmd=showCamps : : -: : : : : 09.0&authredirlevel=1370955451.0&cmd=showCamp&cid=5477450 : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : type=campdate : : : -: : : : : d=7 : : -: : : : : 47.0&cmd=showCamp&cid=4567586 : : -: : : : : 6 : : -: : : : : OqWshMF.pl?cmd=showCamps&emedia_camps=0&ulogin=lena-san-std : : -: : : : : n= : : -: : : : : 82.0&cmd=showCamp&cid=2753896 : : -: : : : : 5 : : -: : : : 75 : : : -: : : : : &mediaType=text : : -: : : : : 0 : : -: : : : : 2 : : -: : : : 19 : : : -: : : : 93 : : : -: : : : : 9 : : -: : : : : dirlevel=1373869482.0&cmd=showCamp&cid=2997085 : : -: : : : : 3 : : -: : : : 21 : : : -: : : : : 88.0 : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 01.0&cmd=showCamp&cid=4567586 : : -: : : : howCampStat&cid=7056273&offline_stat=0&online_stat=1&stat_periods= : 56273&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y2= : : -: : : : 2013-05-25:2013-06-24,2013-05-22:2013-05-23,2013-05-17:2013-05-18& : 13&group=day : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : : -: : : : &goals=0&target_all=yes : : : -: : : : ks=1 : : : -: : : : : 1 : : -: : : : : 1 : : -: : : : : 7 : : -: : : : =total&target_all=1&page=1&cmd=showCampStat&cid=3016175&show_banne : owCampStat&cid=3016175&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : rs_stat=1 : 12-02-03:2012-02-04,2011-08-01:2011-09-04,2010-12-29:2011-01-02&typ : : -: : : : : es=total&page=&group=day&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&sh : : -: : : : : ow_banners_stat=1 : : -: : : : 37 : : : -: : : : 65 : : : -: : : : 01 : : : -: : : : : 5 : : -: : : : : 9 : : -: : : : 19 : : : -: : : : 71 : : : -: : : : : 1 : : -: : : : 08 : : : -: : : : 81 : : : -: : : : : 1 : : -: : : : 45 : : : -: : : : : 0 : : -: : : : : 5 : : -: : : : : 8 : : -: : : : type=campdate : : : -: : : : 90 : gn_direct_plastik_den : : -: : : : : 3 : : -: : : : 40 : : : -: : : : 39 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 03 : : : -: : : : : 6 : : -: : : : ks=1 : : : -: : : : : 7 : : -: : : : : 2 : : -: : : : howCamp&cid=5940082 : : : -: : : : : 9 : : -: : : : 7-.31620-2.1373861604902917.198343986.198343986..64036780.1406.9b1 : 7 : : -: : : : : 1 : : -: : : : : Informers.html : : -: : : : d : : : -: : : : 29 : : : -: : : : 95 : : : -: : : : ransfer : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6870158 : : -: : : : 68 : : : -: : : : : dirlevel=1342185655.0&cmd=showCamps&ulogin=beautyland-artha8 : : -: : : : : 1 : : -: : : : : 8 : : -: : : : 71 : : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 4 : : -: : : : : 018058&cmd=showCamp : : -: : : : 76 : owCampStat&cid=5109476&detail=Yes&types=days : : -: : : : : 9 : : -: : : : : owSubClientCamps : : -: : : : : 1 : : -: : : : : 0 : : -: : : : type=campdate : : : -: : : : 91 : : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=4718384 : : -: : : : : w.advertising.yandex.ru/welcome/pdf/direct_booklet.pdf : : -: : : : : =Yes&types=days&cid=7114664 : : -: : : : : 7 : : -: : : : : &mediaType=text : : -: : : : : 51.0 : : -: : : : ditBannerEasy&cid=7080496&bid=188854799 : 080496&cmd=showCamp : : -: : : : : 8 : : -: : : : 56 : owCampStat&cid=5895656&detail=Yes&types=days : : -: : : : edirlevel=1373786750.0&cmd=showCampStat&cid=6421801&cid=6424294&ci : : : -: : : : d=6487283&cid=6493829&cid=6494462&cid=6495189&stat_type=campdate&c : : : -: : : : srf_token=CzqM8_OaIwxa28-a : : : -: : : : 28 : : : -: : : : : 6 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : type=campdate : : : -: : : : 77 : lshoj-kvartiry : : -: : : : 83 : : : -: : : : : vancedForecast : : -: : : : howCamp&cid=2925555 : owCamp&cid=2925555 : : -: : : : howCamps&ulogin=aplarru : 9&ulogin=aplarru : : -: : : : : 1 : : -: : : : 16 : : : -: : : : : ger : : -: : : : : 52.0&authredirlevel=1373884890.0 : : -: : : : l=Yes&types=days&cid=7026863 : : : -: : : : : 4 : : -: : : : : 5 : : -: : : : 43 : : : -: : : : : 1 : : -: : : : : 9 : : -: : : : : owCamp&cid=7129447#193577203 : : -: : : : 38 : 46.0 : : -: : : : : 8 : : -: : : : 44 : : : -: : : : : 8 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6486629 : : -: : : : : 65.0 : : -: : : : : =Yes&types=days&cid=7131552 : : -: : : : : 3 : : -: : : : howCamp&cid=7129026&tab=wait : owCamp&cid=7129026&tab=active : : -: : : : : 3 : : -: : : : : =Yes&types=days&cid=7015594 : : -: : : : 48 : : : -: : : : : 5 : : -: : : : : 3 : : -: : : : 78 : : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=5826713 : : : -: : : : : 5 : : -: : : : : 1 : : -: : : : : =Yes&types=days&cid=5309194 : : -: : : : howCamps : owCamps : : -: : : : : 6 : : -: : : : 76 : : : -: : : : 21 : : : -: : : : 55 : : : -: : : : : 5 : : -: : : : : 1 : : -: : : : : 8 : : -: : : : : 6 : : -: : : : : 6 : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : : -: : : : : type=campdate : : -: : : : t : : : -: : : : l=Yes&types=days&cid=6752443 : 52443&phrasedate=Yes&target_all=1 : : -: : : : howCamp&cid=6501536&tab=&page=2 : : : -: : : : : 8 : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=7026863 : : -: : : : 41 : : : -: : : : : 5 : : -: : : : : owCamps&ulogin=super-svictor : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6411871 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6189242 : : -: : : : l=Yes&cid=6009684&showCampStat=cmd&types=days&target_all=1&d1=08&m : owCampStat&cid=6009684&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : 1=07&y1=13&d2=08&m2=07&y2=13&group=day : 13-02-25:2013-02-25,2013-02-26:2013-02-26,&detail=yes&page=&y1=2013 : : -: : : : : &m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : 1&mediaType=text : : : -: : : : : =Yes&types=days&cid=6752443 : : -: : : : 42 : : : -: : : : : 3 : : -: : : : howCamps : 2 : : -: : : : 263.0&cmd=showCamp&cid=5830159 : owCampStat&cid=5830159&detail=Yes&types=days : : -: : : : : 7 : : -: : : : : =Yes&types=days&cid=6467205 : : -: : : : : =Yes&types=days&cid=6467203 : : -: : : : 89 : : : -: : : : howCamp&cid=5088603 : owCamp&cid=5088603 : : -: : : : 77 : : : -: : : : : 6 : : -: : : : : 78.0 : : -: : : : 42 : : : -: : : : 38 : owCampStat&cid=6356238&detail=Yes&types=days : : -: : : : howCamps : 6 : : -: : : : : 9 : : -: : : : howCamps&ulogin=discoverycruise : =Yes&types=days&cid=7065506&ulogin=discoverycruise : : -: : : : : 33.0&cmd=showCampStat&detail=Yes&types=days&cid=6393197 : : -: : : : : 7 : : -: : : : ditCamp&cid=5309194 : =Yes;types=days;cid=5309194 : : -: : : : : owCamp&cid=6980076 : : -: : : : : dirlevel=1373863808.0&authredirlevel=1373522397.0&cmd=showCamp&cid= : : -: : : : : 6051926&tab=active : : -: : : : : =easy&from=metrika : : -: : : : : =Yes&types=days&cid=7135716 : : -: : : : : 3 : : -: : : : howCamp&cid=7129447&tab=wait : owCamp&cid=7129447&tab=active : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : 5279956&tab=media : owCamp&cid=5279956&tab=active : : -: : : : ditBannerEasy&cid=6835814&bid=171781249 : 835814&cmd=showCamp : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6931378 : : -: : : : : ype=campdate : : -: : : : 68 : : : -: : : : 42 : : : -: : : : l=Yes&types=days&cid=4718384 : : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 7 : : -: : : : : owCamps&ulogin=visacity08 : : -: : : : : 109741&cmd=showCamp : : -: : : : type=campdate&&ulogin=vnukovo-piter : owCamps&ulogin=vnukovo-piter : : -: : : : : 1 : : -: : : : 3243479&tab=media : owCamp&cid=3243479&tab=active : : -: : : : : 1 : : -: : : : ditBannerEasy&cid=7123828&bid=193277936 : 123828&cmd=showCamp : : -: : : : : 6 : : -: : : : : 0 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : howCamp&cid=7133264 : owCamp&cid=7133264&tab=wait : : -: : : : : 5 : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : howCamps&ulogin=well-mir-ru : 0&ulogin=well-mir-ru : : -: : : : 66 : : : -: : : : 83 : : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : 5 : : -: : : : : 4 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : : owCamps&ulogin=modul-sdskbeton : : -: : : : : =Yes&types=days&cid=7064245 : : -: : : : &ulogin= : : : -: : : : : 3 : : -: : : : howCampStat&cid=3370690&detail=Yes&types=days : 0 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : : 3 : : -: : : : : owCamps : : -: : : : : 7 : : -: : : : : come=yes : : -: : : : : 3 : : -: : : : : 9 : : -: : : : : 3 : : -: : : : : 6 : : -: : : : howCampStat&cid=7109552&detail=Yes&types=days : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7109552&sh : : -: : : : : ow_banners_stat=1 : : -: : : : howCamp&cid=7134723 : : : -: : : : howCamps&ulogin=alfenjoy-dir : 3&ulogin=alfenjoy-dir : : -: : : : howCamps&ulogin=allstarshop-yd : owCamps&ulogin=allstarshop-yd : : -: : : : howCamps : 7 : : -: : : : : 6 : : -: : : : 35 : : : -: : : : 96&ulogin=kuncevo-ru : vo-avto-obyavlyaet-start-prodazh-LEXUS-ES-i-LEXUS-RX-spetsialnyh-ve : : -: : : : : rsiy.tmex : : -: : : : : =Yes&types=days&cid=6144446 : : -: : : : : 2 : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 137413&cmd=orderCamp&agree=yes : : : -: : : : : 8 : : -: : : : 80 : : : -: : : : 994219&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y : 9 : : -: : : : 2=13&group=day : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=3346173 : : -: : : : : =Yes&types=days&cid=4300226 : : -: : : : 90 : : : -: : : : id=7 : : : -: : : : : owCampStat&cid=4194803&cid=5256554&cid=5718709&cid=6790267&cid=6790 : : -: : : : : 271&cid=7070376&stat_type=campdate&ulogin=mybalcony-ru&csrf_token=L : : -: : : : : BcSJLrXLKh9zXYq : : -: : : : 38 : owCampStat&cid=6356238&detail=Yes&types=days : : -: : : : 431468&phrasedate=Yes&target_all=1 : owCampStat&cid=5431468&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-06-27:2013-07-10,2013-01-01:2013-06-01,2013-06-01:2013-07-03&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 3 : : -: : : : 74 : : : -: : : : all : : : -: : : : : 7 : : -: : : : : 0 : : -: : : : 16 : : : -: : : : : owClients : : -: : : : : 6 : : -: : : : : =Yes&types=days&cid=1475303 : : -: : : : : 2 : : -: : : : l=Yes&types=days&cid=6942653 : owCampStat&cid=6942653&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-06-22:2013-06-22,2013-06-16:2013-06-18,2013-06-17:2013-06-18&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : 97 : : : -: : : : howCamps : 6 : : -: : : : 51 : : : -: : : : : 480202&cmd=showCamp : : -: : : : howCamp&cid=3973331&ulogin=kuncevo-ru : : : -: : : : : owCamps&ulogin=visamaster08 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=6800439 : : : -: : : : 09 : owCamp&cid=6797009 : : -: : : : : 9 : : -: : : : : dirlevel=1373859813.0&cmd=showCampStat&cid=5538008&detail=Yes&types : : -: : : : : =days : : -: : : : 316.0&cmd=showCamp&cid=5993938 : : : -: : : : : 2 : : -: : : : : 7 : : -: : : : : =Yes&types=days&cid=7103125 : : -: : : : : 8 : : -: : : : howCamp&cid=7129447&tab=wait : owCamp&cid=7129447&tab=active : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6883763 : : -: : : : : 7 : : -: : : : : ype=campdate : : -: : : : : =Yes&types=days&cid=1724608 : : -: : : : : 5 : : -: : : : : 2 : : -: : : : : 3 : : -: : : : : 2 : : -: : : : 09 : : : -: : : : : 4 : : -: : : : : ed : : -: : : : = : 17.0&authredirlevel=1373534053.0&cmd=apiApplicationList&ulogin= : : -: : : : : 2 : : -: : : : HB.31620-2.1373870178024287.199046816.199046816..14150262.1406.c65 : : : -: : : : &rnd=0.9338765616586568 : : : -: : : : 368.0&cmd=showCamp&cid=4685945 : 74.0&authredirlevel=1363929368.0&cmd=showCamp&cid=4685945 : : -: : : : : 5 : : -: : : : : 97.0&cmd=ForecastByWords : : -: : : : 71 : : : -: : : : howCampStat&cid=2196876&detail=Yes&types=days : 6 : : -: : : : = : : : -: : : : howCampStat&cid=7056273&offline_stat=0&online_stat=1&stat_periods= : 56273&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y2= : : -: : : : 2013-05-25:2013-06-24,2013-05-22:2013-05-23,2013-05-17:2013-05-18& : 13&group=day : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : : -: : : : &goals=0&target_all=yes : : : -: : : : : 9 : : -: : : : edirlevel=1373868161.0&&cid=7027214&cmd=showCamp : owCampStat&cid=7027214&detail=Yes&types=days : : -: : : : : =Yes&types=days&cid=6630335 : : -: : : : : owCamps : : -: : : : nt_country=225&cmd=editBannerEasy¤cy=YND_FIXED&interface_typ : 136912&cmd=showCamp&unglued=194242390 : : -: : : : e=easy¬new=yes : : : -: : : : : 7 : : -: : : : : 6 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-11:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-03:201 : : -: : : : 013-07-15,2013-07-01:2013-07-14,2013-07-05:2013-07-13&stat_type=ca : 3-07-15,2013-07-11:2013-07-15,2013-07-01:2013-07-14&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&target_al : ate&y1=2013&m1=07&d1=03&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : l=1 : : : -: : : : : 83.0 : : -: : : : howCamps&ulogin=nordex-grandit : 57.0&cmd=showCampStat&detail=Yes&types=days&cid=6947313&ulogin=nord : : -: : : : : ex-grandit : : -: : : : : =Yes&types=days&cid=7068093 : : -: : : : : =Yes&types=days&cid=6834748 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2011-10-27:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2011-10-27:201 : : -: : : : 011-10-27,2011-08-31:2011-08-31,2011-08-26:2011-08-26&stat_type=ca : 1-10-27,2011-08-31:2011-08-31,2011-08-26:2011-08-26&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=14&y2=2013&m2=07&d2=14&group=day&target_al : ate&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : l=1 : : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 38 : 07.0&authredirlevel=1373876446.0 : : -: : : : : 134782&cmd=showCamp&fixated=150055634 : : -: : : : : dirlevel=1364366494.0&cmd=showCamp&cid=6106774 : : -: : : : : =Yes&types=days&cid=7072622 : : -: : : : : owClients : : -: : : : : 113103&cmd=showCamp : : -: : : : : 2 : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : : 5 : : -: : : : : 05.6&authredirlevel=1372014905.5&authredirlevel=1372014905.4&authre : : -: : : : : dirlevel=1372014905.3&authredirlevel=1372014905.2&authredirlevel=13 : : -: : : : : 72014905.1&authredirlevel=1372014905.0&authredirlevel=1370480737.6& : : -: : : : : authredirlevel=1370480737.5&authredirlevel=1370480737.4&authredirle : : -: : : : : vel=1370480737.3&authredirlevel=1370480737.2&authredirlevel=1370480 : : -: : : : : 737.1&authredirlevel=1370480737.0&cmd=showCampStat&stat_type=campda : : -: : : : : te&target_0=1&target_1=1&target_all=1 : : -: : : : : ulogin= : : -: : : : : 5 : : -: : : : howCamp&cid=6980076 : owCampStat&cid=6980076&detail=Yes&types=days : : -: : : : &ulogin= : ulogin= : : -: : : : howClients : owCamps&ulogin=K-PV-beeline : : -: : : : l=Yes&types=days&cid=88689 : : : -: : : : 32 : : : -: : : : 15 : owCampStat&cid=5991615&detail=Yes&types=days : : -: : : : howCamp&cid=6980076 : owCamp&cid=6980076 : : -: : : : : =Yes&types=days&cid=5871623 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 8 : : -: : : : : owCampStat&cid=4917291&detail=Yes&types=days : : -: : : : : 2 : : -: : : : : 90.0 : : -: : : : : 8 : : -: : : : : owCamps&ulogin=domusmebel-ru : : -: : : : : 0 : : -: : : : : =Yes&types=days&cid=4735372 : : -: : : : : 9 : : -: : : : 49 : : : -: : : : : 5 : : -: : : : 08 : : : -: : : : howCampStat&cid=3353427&offline_stat=0&online_stat=0&stat_periods= : d2=15&target_0=0&stat_periods=2013-07-01:2013-07-06,2013-06-27:2013 : : -: : : : 2013-07-01:2013-07-06,2013-06-27:2013-07-05,2013-06-01:2013-06-30& : -07-05,2013-06-01:2013-06-30&m2=07&stat_type=custom&onpage=100&grou : : -: : : : stat_type=custom&sort=&reverse=ARRAY(0x1639e200)&group_by=tag&filt : p_by=tag, banner, page, phrase&d1=15&target_1=0&cmd=showCampStat&ta : : -: : : : er_tag=&group_by=banner&filter_banner=&group_by=page&filter_page=& : rget_all=1&online_stat=0&m1=07&cid=3353427&goals=0&offline_stat=0&y : : -: : : : filter_page_target=&group_by=phrase&filter_phrase=&goals=0&y1=2013 : 2=13&sort=clicks&reverse=1 : : -: : : : &m1=07&d1=15&y2=2013&m2=07&d2=15&onpage=100 : : : -: : : : : owCamps : : -: : : : : 2 : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 8 : : -: : : : 44 : owCampStat&cid=4154744&detail=Yes&types=days : : -: : : : 01 : : : -: : : : site_promotion_search : : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 02.0 : : -: : : : : 7 : : -: : : : howCamp&cid=6522990&tab=off : owCamp&cid=6522990 : : -: : : : : 9 : : -: : : : : 8 : : -: : : : : 9 : : -: : : : : 5 : : -: : : : : 4 : : -: : : : 813.0&cmd=showCamp&cid=5210661 : : : -: : : : : 63.0&cmd=showCamp&cid=5830159 : : -: : : : : =Yes&types=days&cid=6993362 : : -: : : : : vancedForecast : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : 7090667&cmd=showCamp : owCampStat&cid=7090667&detail=Yes&types=days : : -: : : : 77&ulogin=svk-direct : : : -: : : : : 3 : : -: : : : : 1 : : -: : : : howCampStat&cid=5975585&detail=Yes&types=days : 5 : : -: : : : : 8 : : -: : : : : 1 : : -: : : : : 41.0&authredirlevel=1373879619.0&cmd=showCamp&cid=5356346 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-03-01:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-03-01:201 : : -: : : : 013-07-15,2013-06-01:2013-06-14,2013-06-01:2013-07-15&stat_type=ca : 3-07-15,2013-06-01:2013-06-14,2013-06-01:2013-07-15&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=month&target_ : ate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : all=1 : : : -: : : : : owCamps&ulogin=infoprom51 : : -: : : : : 64.0&cmd=showCamp&cid=5028648 : : -: : : : 64 : : : -: : : : : 6 : : -: : : : : 34.0&cmd=ForecastByWords : : -: : : : : 4 : : -: : : : ts : : : -: : : : : 2 : : -: : : : ncy : : : -: : : : : 8 : : -: : : : : 55.0&cmd=showCampStat&detail=Yes&types=days&cid=6849113 : : -: : : : howCamps&ulogin=aplarru : 9&ulogin=aplarru : : -: : : : : =Yes&types=days&cid=5039376 : : -: : : : : owCamp&cid=4748947&ulogin=cont5nn#71488518 : : -: : : : _camps=0&cmd=showSubClientCamps&authredirlevel=1373873308.0&settin : dirlevel=1373880608.0&media_camps=0&cmd=showSubClientCamps&authredi : : -: : : : gs_sorted=&sort=total&reverse=1 : rlevel=1373873308.0&settings_sorted=&sort=total&reverse=0 : : -: : : : howCamp&cid=4581573&tab=decline : owCamp&cid=4581573&tab=active : : -: : : : : 9 : : -: : : : howClients : vancedForecast : : -: : : : : 983763&cmd=showCamp&unglued=194305392 : : -: : : : : 9 : : -: : : : : 5 : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=7030009 : : -: : : : 0 : : : -: : : : : 8 : : -: : : : : 9 : : -: : : : : 65.0&cmd=showCamp&cid=7135509 : : -: : : : 642.0 : ype=campdate : : -: : : : : 5 : : -: : : : 35 : 5 : : -: : : : : dirlevel=1373877895.0&cmd=showCamp&cid=3055906 : : -: : : : : 6 : : -: : : : 099442&cid=7100140&cid=7100144&cid=7100148&cid=7100150&cid=7132172 : owCampStat&cid=7099442&cid=7100140&cid=7100144&cid=7100148&cid=7100 : : -: : : : &cid=7132174&cid=7132175&cid=7132177&cid=7132179&cid=7132632&cid=7 : 150&cid=7132172&cid=7132174&cid=7132175&cid=7132177&cid=7132179&cid : : -: : : : 132838&cid=7132888&cid=7133059&cid=7133217&stat_type=campdate&csrf : =7132632&cid=7132838&cid=7132888&cid=7133059&cid=7133217&offline_st : : -: : : : _token=aENmaxzslTI5oMqD : at=0&online_stat=0&stat_periods=2013-07-01:2013-07-13,2013-07-01:20 : : -: : : : : 13-07-12,2013-06-14:2013-07-11&stat_type=campdate&y1=2013&m1=07&d1= : : -: : : : : 15&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : 35 : 63.0&authredirlevel=1373873632.0 : : -: : : : : 1 : : -: : : : 73 : : : -: : : : howCampStat&cid=6981382&offline_stat=0&online_stat=1&stat_periods= : owCampStat&cid=6981382&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : 2013-06-10:2013-06-25,2013-06-10:2013-06-24,&detail=yes&page=&y1=2 : 13-06-10:2013-06-25,2013-06-10:2013-06-24,&detail=yes&page=&y1=2013 : : -: : : : 013&m1=07&d1=05&y2=2013&m2=07&d2=12&group=day&goals=0&target_all=y : &m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : es : : : -: : : : : =Yes&types=days&cid=6621650 : : -: : : : gSearch&who=camps&cid=&camp_login=&camp_name=атлант : 4&ulogin=atlant-b2bnew : : -: : : : : 0 : : -: : : : : owCamps&ulogin=domusmebel-ru : : -: : : : : 3 : : -: : : : : owCamp&cid=3601040 : : -: : : : : =Yes&types=days&cid=6858643 : : -: : : : : =Yes&types=days&cid=7109462 : : -: : : : : =Yes&types=days&cid=7085888 : : -: : : : 25 : : : -: : : : : owCamps : : -: : : : d=18770681&resp_desc=success : owCamps : : -: : : : howCamp&cid=6980076 : owCampStat&cid=6980076&detail=Yes&types=days : : -: : : : : 5 : : -: : : : : 4 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=5129796 : : -: : : : : 7 : : -: : : : howCampStat&cid=7104084&detail=Yes&types=days : 104084&cmd=showCamp : : -: : : : : 6 : : -: : : : : 2 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=3427083 : : : -: : : : : =Yes&types=days&cid=7060114 : : -: : : : l=Yes&types=days&cid=6833696 : owCampStat&cid=6833696&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-20:2013-06-21,&detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m : : -: : : : : 2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : : 137972&cmd=showCamp&unglued=194336987 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : &mediaType=text&firstbanner=std : : -: : : : 137830&cmd=orderCamp&agree=yes : : : -: : : : : 9 : : -: : : : 09 : : : -: : : : : 131075&cmd=showCamp : : -: : : : : 2 : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6960697 : : -: : : : 7129022&cmd=showCamp : owCampStat&cid=7129022&detail=Yes&types=days : : -: : : : 43 : : : -: : : : : 8 : : -: : : : 17 : owCamp&cid=7082517 : : -: : : : : 6 : : -: : : : 31&ulogin=kuncevo-ru : : : -: : : : 394451&phrasedate=Yes&target_all=1 : owCampStat&cid=4394451&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 12-01-01:2012-11-21,2012-01-01:2012-07-31,2012-03-23:2012-03-23&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 1 : : -: : : : howCamps : owCamps : : -: : : : : 1 : : -: : : : : 6 : : -: : : : gSearch&who=camps&cid=&camp_login=&camp_name=дизв : 9&ulogin=dizvitr-b2b : : -: : : : : 7 : : -: : : : : 9 : : -: : : : : campdate : : -: : : : : 0 : : -: : : : : 6 : : -: : : : : 6 : : -: : : : howCamps&ulogin=oknabuduschego : =Yes&types=days&cid=5506368&ulogin=oknabuduschego : : -: : : : howClients&sort=total&reverse=0 : owCamps&ulogin=Rusokon2007 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=4879898 : : -: : : : 1&mediaType=text : : : -: : : : 85 : : : -: : : : howCamps&ulogin=visacity08 : 5&ulogin=visacity08 : : -: : : : 56 : : : -: : : : : owCamps&ulogin=dc12a : : -: : : : 59 : : : -: : : : howCamps&ulogin=ul-smilestar-su : =ul-smilestar-su#list : : -: : : : howCampStat&cid=7053221&detail=Yes&types=days : owCampStat&cid=7053221&offline_stat=0&online_stat=0&stat_periods=&d : : -: : : : : etail=yes&page=&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&g : : -: : : : : oals=0&target_all=yes&show_banners_stat=1 : : -: : : : : =Yes&types=days&cid=7133318 : : -: : : : : 5 : : -: : : : 25 : : : -: : : : howCamp&tab=all&cid=7126549&ulogin=transkontinental2014#194270034 : : : -: : : : : 2 : : -: : : : 02 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 07 : : : -: : : : : 1 : : -: : : : : 9 : : -: : : : howCamp&cid=7078898&tab=off : owCamp&cid=7078898 : : -: : : : 01 : : : -: : : : : 4 : : -: : : : : 35.0 : : -: : : : : owCampStat&cid=2886087&detail=Yes&types=days : : -: : : : : 8 : : -: : : : type=campdate : dirlevel=1373865822.0&cmd=showCampStat&offline_stat=0&online_stat=0 : : -: : : : : &stat_periods=2013-07-04:2013-07-05,2013-06-10:2013-06-13,2013-05-2 : : -: : : : : 1:2013-05-31&stat_type=campdate&y1=2013&m1=07&d1=15&y2=2013&m2=07&d : : -: : : : : 2=15&group=day&target_all=1 : : -: : : : : 0 : : -: : : : : owCamp&cid=6354573#194189392 : : -: : : : : owCamps : : -: : : : 25 : : : -: : : : : 8 : : -: : : : : 04.0&cmd=showCamp&cid=7129428 : : -: : : : howCamp&cid=7130323 : ids=193775525&cid=7130323 : : -: : : : howCamps&ulogin=richton-studio : 7&ulogin=richton-studio : : -: : : : : owCampStat&cid=7034406&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-07-11:2013-07-11,2013-07-12:2013-07-12,2013-07-13:2013-07-13&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 0 : : -: : : : howCamp&tab=decline&cid=6886514 : owCamp&cid=6886514&tab=active : : -: : : : : 33.0 : : -: : : : : 60.0 : : -: : : : : 5 : : -: : : : : dirlevel=1373866480.0&cmd=showCampStat&stat_type=by_agency_clients : : -: : : : : 9 : : -: : : : : 463697&cmd=showCamp : : -: : : : 28 : : : -: : : : : 6 : : -: : : : : 8 : : -: : : : : 4 : : -: : : : 60 : : : -: : : : : 5 : : -: : : : : 7 : : -: : : : : 1 : : -: : : : howCampStat&cid=6456112&offline_stat=0&online_stat=0&stat_periods= : owCampStat&cid=6456112&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : 2013-07-10:2013-07-10,2013-07-09:2013-07-09,2013-07-08:2013-07-08& : 13-07-10:2013-07-10,2013-07-09:2013-07-09,2013-07-08:2013-07-08&phr : : -: : : : phrasedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : =day&goals=0&target_all=yes : &goals=0&target_all=yes : : -: : : : : 7 : : -: : : : : 3 : : -: : : : howCampStat&cid=4121349&detail=Yes&types=days : owCampStat&cid=4121349&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-07-08:2013-07-09,2013-06-01:2013-06-06,2013-04-01:2013-06-03&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : : owClients : : -: : : : 15 : : : -: : : : l=Yes&types=days&cid=7103094 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7103094&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 4 : : -: : : : 84 : : : -: : : : 89 : : : -: : : : 1&mediaType=text : : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 2 : : -: : : : 86 : : : -: : : : : 4 : : -: : : : 12 : : : -: : : : : 6 : : -: : : : : owCamps&ulogin=dc12a : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 9 : : -: : : : l=Yes&types=days&cid=141133 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=141133&sho : : -: : : : : w_banners_stat=1 : : -: : : : : 108927&cmd=showCamp : : -: : : : howCamp&cid=6886521&tab=off : owCamp&cid=6886521 : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=7133822 : : -: : : : : owCamps&ulogin=mitsubishi-galant2013 : : -: : : : ditCamp&cid=7088102 : owCamp&cid=7088102 : : -: : : : l=Yes&types=days&cid=5033617 : : : -: : : : : 2 : : -: : : : 56 : : : -: : : : : =Yes&types=days&cid=5056684 : : -: : : : 25 : : : -: : : : : owCamps : : -: : : : l=Yes&types=days&cid=6479409 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=6479409&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 9 : : -: : : : : 85.0 : : -: : : : : =Yes&types=days&cid=7133333 : : -: : : : : owCampStat&cid=3618668&detail=Yes&types=days : : -: : : : 74 : owCamp&cid=4275274&tag=303211 : : -: : : : : =Yes&types=days&cid=3298907 : : -: : : : : 9 : : -: : : : : 1 : : -: : : : howCamp&cid=5931713&tab=wait : owCamp&cid=5931713&tab=active : : -: : : : : &mediaType=text : : -: : : : 22 : : : -: : : : 002.0 : 7 : : -: : : : : e_stat=0&online_stat=0&stat_type=campdate&group=day&target_0=1&targ : : -: : : : : et_1=1 : : -: : : : : 4 : : -: : : : mportCampSuccess&cid=4202002&make_new_camp=0 : owCamp&cid=4202002 : : -: : : : : 5 : : -: : : : 87 : : : -: : : : howCamps : owCamps : : -: : : : 99 : : : -: : : : : 3 : : -: : : : 90 : 0 : : -: : : : edirlevel=1373611946.0&authredirlevel=1372140313.0&authredirlevel= : : : -: : : : 1371447906.0&authredirlevel=1368426548.0&authredirlevel=1367818443 : : : -: : : : .0&cmd=showCamps&ulogin=Newdesign-Larus : : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : 97 : : : -: : : : : 9 : : -: : : : howCamps&ulogin=ogurcova-mkt : =Yes&types=days&cid=4139609&ulogin=ogurcova-mkt : : -: : : : : 2 : : -: : : : : 2 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : : 136655&cmd=showCamp : : -: : : : : 8 : : -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -1000 tuples (4.6s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageVi ews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -: icso : chen : ngin : : : iews : -: urce : gine : eid : : : : -: id : id : : : : : -+======+======+======+====================================================================+=====================================================================+=======+ -: : : : : ype=campdate : : -: : : : type=campdate : : : -: : : : : &payment_type=1 : : -: : : : : d=7 : : -: : : : : &mediaType=text : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0 : : : -: : : : : 79.0&authredirlevel=1373600157.0&authredirlevel=1373081665.0&authre : : -: : : : : dirlevel=1372822331.0&authredirlevel=1372563095.0&authredirlevel=13 : : -: : : : : 72303801.0&authredirlevel=1372044532.0&authredirlevel=1371730939.0& : : -: : : : : authredirlevel=1371471570.0&authredirlevel=1371212358.0&cmd=showCam : : -: : : : : p&cid=2856748 : : -: : : : t : : : -: : : : : ype=campdate : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : -: : : : : s=1 : : -: : : : : owCamps : : -: : : : 1&mediaType=text : : : -: : : : : dirlevel=1373731047.0&cmd=showCamps : : -: : : : : 09.0&authredirlevel=1370955451.0&cmd=showCamp&cid=5477450 : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : type=campdate : : : -: : : : : d=7 : : -: : : : : 47.0&cmd=showCamp&cid=4567586 : : -: : : : : 6 : : -: : : : : OqWshMF.pl?cmd=showCamps&emedia_camps=0&ulogin=lena-san-std : : -: : : : : n= : : -: : : : : 82.0&cmd=showCamp&cid=2753896 : : -: : : : : 5 : : -: : : : 75 : : : -: : : : : &mediaType=text : : -: : : : : 0 : : -: : : : : 2 : : -: : : : 19 : : : -: : : : 93 : : : -: : : : : 9 : : -: : : : : dirlevel=1373869482.0&cmd=showCamp&cid=2997085 : : -: : : : : 3 : : -: : : : 21 : : : -: : : : : 88.0 : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 01.0&cmd=showCamp&cid=4567586 : : -: : : : howCampStat&cid=7056273&offline_stat=0&online_stat=1&stat_periods= : 56273&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y2= : : -: : : : 2013-05-25:2013-06-24,2013-05-22:2013-05-23,2013-05-17:2013-05-18& : 13&group=day : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : : -: : : : &goals=0&target_all=yes : : : -: : : : ks=1 : : : -: : : : : 1 : : -: : : : : 1 : : -: : : : : 7 : : -: : : : =total&target_all=1&page=1&cmd=showCampStat&cid=3016175&show_banne : owCampStat&cid=3016175&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : rs_stat=1 : 12-02-03:2012-02-04,2011-08-01:2011-09-04,2010-12-29:2011-01-02&typ : : -: : : : : es=total&page=&group=day&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&sh : : -: : : : : ow_banners_stat=1 : : -: : : : 37 : : : -: : : : 65 : : : -: : : : 01 : : : -: : : : : 5 : : -: : : : : 9 : : -: : : : 19 : : : -: : : : 71 : : : -: : : : : 1 : : -: : : : 08 : : : -: : : : 81 : : : -: : : : : 1 : : -: : : : 45 : : : -: : : : : 0 : : -: : : : : 5 : : -: : : : : 8 : : -: : : : type=campdate : : : -: : : : 90 : gn_direct_plastik_den : : -: : : : : 3 : : -: : : : 40 : : : -: : : : 39 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 03 : : : -: : : : : 6 : : -: : : : ks=1 : : : -: : : : : 7 : : -: : : : : 2 : : -: : : : howCamp&cid=5940082 : : : -: : : : : 9 : : -: : : : 7-.31620-2.1373861604902917.198343986.198343986..64036780.1406.9b1 : 7 : : -: : : : : 1 : : -: : : : : Informers.html : : -: : : : d : : : -: : : : 29 : : : -: : : : 95 : : : -: : : : ransfer : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6870158 : : -: : : : 68 : : : -: : : : : dirlevel=1342185655.0&cmd=showCamps&ulogin=beautyland-artha8 : : -: : : : : 1 : : -: : : : : 8 : : -: : : : 71 : : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 4 : : -: : : : : 018058&cmd=showCamp : : -: : : : 76 : owCampStat&cid=5109476&detail=Yes&types=days : : -: : : : : 9 : : -: : : : : owSubClientCamps : : -: : : : : 1 : : -: : : : : 0 : : -: : : : type=campdate : : : -: : : : 91 : : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=4718384 : : -: : : : : w.advertising.yandex.ru/welcome/pdf/direct_booklet.pdf : : -: : : : : =Yes&types=days&cid=7114664 : : -: : : : : 7 : : -: : : : : &mediaType=text : : -: : : : : 51.0 : : -: : : : ditBannerEasy&cid=7080496&bid=188854799 : 080496&cmd=showCamp : : -: : : : : 8 : : -: : : : 56 : owCampStat&cid=5895656&detail=Yes&types=days : : -: : : : edirlevel=1373786750.0&cmd=showCampStat&cid=6421801&cid=6424294&ci : : : -: : : : d=6487283&cid=6493829&cid=6494462&cid=6495189&stat_type=campdate&c : : : -: : : : srf_token=CzqM8_OaIwxa28-a : : : -: : : : 28 : : : -: : : : : 6 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : type=campdate : : : -: : : : 77 : lshoj-kvartiry : : -: : : : 83 : : : -: : : : : vancedForecast : : -: : : : howCamp&cid=2925555 : owCamp&cid=2925555 : : -: : : : howCamps&ulogin=aplarru : 9&ulogin=aplarru : : -: : : : : 1 : : -: : : : 16 : : : -: : : : : ger : : -: : : : : 52.0&authredirlevel=1373884890.0 : : -: : : : l=Yes&types=days&cid=7026863 : : : -: : : : : 4 : : -: : : : : 5 : : -: : : : 43 : : : -: : : : : 1 : : -: : : : : 9 : : -: : : : : owCamp&cid=7129447#193577203 : : -: : : : 38 : 46.0 : : -: : : : : 8 : : -: : : : 44 : : : -: : : : : 8 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6486629 : : -: : : : : 65.0 : : -: : : : : =Yes&types=days&cid=7131552 : : -: : : : : 3 : : -: : : : howCamp&cid=7129026&tab=wait : owCamp&cid=7129026&tab=active : : -: : : : : 3 : : -: : : : : =Yes&types=days&cid=7015594 : : -: : : : 48 : : : -: : : : : 5 : : -: : : : : 3 : : -: : : : 78 : : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=5826713 : : : -: : : : : 5 : : -: : : : : 1 : : -: : : : : =Yes&types=days&cid=5309194 : : -: : : : howCamps : owCamps : : -: : : : : 6 : : -: : : : 76 : : : -: : : : 21 : : : -: : : : 55 : : : -: : : : : 5 : : -: : : : : 1 : : -: : : : : 8 : : -: : : : : 6 : : -: : : : : 6 : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : : -: : : : : type=campdate : : -: : : : t : : : -: : : : l=Yes&types=days&cid=6752443 : 52443&phrasedate=Yes&target_all=1 : : -: : : : howCamp&cid=6501536&tab=&page=2 : : : -: : : : : 8 : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=7026863 : : -: : : : 41 : : : -: : : : : 5 : : -: : : : : owCamps&ulogin=super-svictor : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6411871 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6189242 : : -: : : : l=Yes&cid=6009684&showCampStat=cmd&types=days&target_all=1&d1=08&m : owCampStat&cid=6009684&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : 1=07&y1=13&d2=08&m2=07&y2=13&group=day : 13-02-25:2013-02-25,2013-02-26:2013-02-26,&detail=yes&page=&y1=2013 : : -: : : : : &m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : 1&mediaType=text : : : -: : : : : =Yes&types=days&cid=6752443 : : -: : : : 42 : : : -: : : : : 3 : : -: : : : howCamps : 2 : : -: : : : 263.0&cmd=showCamp&cid=5830159 : owCampStat&cid=5830159&detail=Yes&types=days : : -: : : : : 7 : : -: : : : : =Yes&types=days&cid=6467205 : : -: : : : : =Yes&types=days&cid=6467203 : : -: : : : 89 : : : -: : : : howCamp&cid=5088603 : owCamp&cid=5088603 : : -: : : : 77 : : : -: : : : : 6 : : -: : : : : 78.0 : : -: : : : 42 : : : -: : : : 38 : owCampStat&cid=6356238&detail=Yes&types=days : : -: : : : howCamps : 6 : : -: : : : : 9 : : -: : : : howCamps&ulogin=discoverycruise : =Yes&types=days&cid=7065506&ulogin=discoverycruise : : -: : : : : 33.0&cmd=showCampStat&detail=Yes&types=days&cid=6393197 : : -: : : : : 7 : : -: : : : ditCamp&cid=5309194 : =Yes;types=days;cid=5309194 : : -: : : : : owCamp&cid=6980076 : : -: : : : : dirlevel=1373863808.0&authredirlevel=1373522397.0&cmd=showCamp&cid= : : -: : : : : 6051926&tab=active : : -: : : : : =easy&from=metrika : : -: : : : : =Yes&types=days&cid=7135716 : : -: : : : : 3 : : -: : : : howCamp&cid=7129447&tab=wait : owCamp&cid=7129447&tab=active : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : 5279956&tab=media : owCamp&cid=5279956&tab=active : : -: : : : ditBannerEasy&cid=6835814&bid=171781249 : 835814&cmd=showCamp : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6931378 : : -: : : : : ype=campdate : : -: : : : 68 : : : -: : : : 42 : : : -: : : : l=Yes&types=days&cid=4718384 : : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 7 : : -: : : : : owCamps&ulogin=visacity08 : : -: : : : : 109741&cmd=showCamp : : -: : : : type=campdate&&ulogin=vnukovo-piter : owCamps&ulogin=vnukovo-piter : : -: : : : : 1 : : -: : : : 3243479&tab=media : owCamp&cid=3243479&tab=active : : -: : : : : 1 : : -: : : : ditBannerEasy&cid=7123828&bid=193277936 : 123828&cmd=showCamp : : -: : : : : 6 : : -: : : : : 0 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : howCamp&cid=7133264 : owCamp&cid=7133264&tab=wait : : -: : : : : 5 : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : howCamps&ulogin=well-mir-ru : 0&ulogin=well-mir-ru : : -: : : : 66 : : : -: : : : 83 : : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : 5 : : -: : : : : 4 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : : owCamps&ulogin=modul-sdskbeton : : -: : : : : =Yes&types=days&cid=7064245 : : -: : : : &ulogin= : : : -: : : : : 3 : : -: : : : howCampStat&cid=3370690&detail=Yes&types=days : 0 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : : 3 : : -: : : : : owCamps : : -: : : : : 7 : : -: : : : : come=yes : : -: : : : : 3 : : -: : : : : 9 : : -: : : : : 3 : : -: : : : : 6 : : -: : : : howCampStat&cid=7109552&detail=Yes&types=days : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7109552&sh : : -: : : : : ow_banners_stat=1 : : -: : : : howCamp&cid=7134723 : : : -: : : : howCamps&ulogin=alfenjoy-dir : 3&ulogin=alfenjoy-dir : : -: : : : howCamps&ulogin=allstarshop-yd : owCamps&ulogin=allstarshop-yd : : -: : : : howCamps : 7 : : -: : : : : 6 : : -: : : : 35 : : : -: : : : 96&ulogin=kuncevo-ru : vo-avto-obyavlyaet-start-prodazh-LEXUS-ES-i-LEXUS-RX-spetsialnyh-ve : : -: : : : : rsiy.tmex : : -: : : : : =Yes&types=days&cid=6144446 : : -: : : : : 2 : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 137413&cmd=orderCamp&agree=yes : : : -: : : : : 8 : : -: : : : 80 : : : -: : : : 994219&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y : 9 : : -: : : : 2=13&group=day : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=3346173 : : -: : : : : =Yes&types=days&cid=4300226 : : -: : : : 90 : : : -: : : : id=7 : : : -: : : : : owCampStat&cid=4194803&cid=5256554&cid=5718709&cid=6790267&cid=6790 : : -: : : : : 271&cid=7070376&stat_type=campdate&ulogin=mybalcony-ru&csrf_token=L : : -: : : : : BcSJLrXLKh9zXYq : : -: : : : 38 : owCampStat&cid=6356238&detail=Yes&types=days : : -: : : : 431468&phrasedate=Yes&target_all=1 : owCampStat&cid=5431468&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-06-27:2013-07-10,2013-01-01:2013-06-01,2013-06-01:2013-07-03&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 3 : : -: : : : 74 : : : -: : : : all : : : -: : : : : 7 : : -: : : : : 0 : : -: : : : 16 : : : -: : : : : owClients : : -: : : : : 6 : : -: : : : : =Yes&types=days&cid=1475303 : : -: : : : : 2 : : -: : : : l=Yes&types=days&cid=6942653 : owCampStat&cid=6942653&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-06-22:2013-06-22,2013-06-16:2013-06-18,2013-06-17:2013-06-18&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : 97 : : : -: : : : howCamps : 6 : : -: : : : 51 : : : -: : : : : 480202&cmd=showCamp : : -: : : : howCamp&cid=3973331&ulogin=kuncevo-ru : : : -: : : : : owCamps&ulogin=visamaster08 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=6800439 : : : -: : : : 09 : owCamp&cid=6797009 : : -: : : : : 9 : : -: : : : : dirlevel=1373859813.0&cmd=showCampStat&cid=5538008&detail=Yes&types : : -: : : : : =days : : -: : : : 316.0&cmd=showCamp&cid=5993938 : : : -: : : : : 2 : : -: : : : : 7 : : -: : : : : =Yes&types=days&cid=7103125 : : -: : : : : 8 : : -: : : : howCamp&cid=7129447&tab=wait : owCamp&cid=7129447&tab=active : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6883763 : : -: : : : : 7 : : -: : : : : ype=campdate : : -: : : : : =Yes&types=days&cid=1724608 : : -: : : : : 5 : : -: : : : : 2 : : -: : : : : 3 : : -: : : : : 2 : : -: : : : 09 : : : -: : : : : 4 : : -: : : : : ed : : -: : : : = : 17.0&authredirlevel=1373534053.0&cmd=apiApplicationList&ulogin= : : -: : : : : 2 : : -: : : : HB.31620-2.1373870178024287.199046816.199046816..14150262.1406.c65 : : : -: : : : &rnd=0.9338765616586568 : : : -: : : : 368.0&cmd=showCamp&cid=4685945 : 74.0&authredirlevel=1363929368.0&cmd=showCamp&cid=4685945 : : -: : : : : 5 : : -: : : : : 97.0&cmd=ForecastByWords : : -: : : : 71 : : : -: : : : howCampStat&cid=2196876&detail=Yes&types=days : 6 : : -: : : : = : : : -: : : : howCampStat&cid=7056273&offline_stat=0&online_stat=1&stat_periods= : 56273&phrasedate=Yes&target_all=1&d1=15&m1=07&y1=13&d2=15&m2=07&y2= : : -: : : : 2013-05-25:2013-06-24,2013-05-22:2013-05-23,2013-05-17:2013-05-18& : 13&group=day : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : : -: : : : &goals=0&target_all=yes : : : -: : : : : 9 : : -: : : : edirlevel=1373868161.0&&cid=7027214&cmd=showCamp : owCampStat&cid=7027214&detail=Yes&types=days : : -: : : : : =Yes&types=days&cid=6630335 : : -: : : : : owCamps : : -: : : : nt_country=225&cmd=editBannerEasy¤cy=YND_FIXED&interface_typ : 136912&cmd=showCamp&unglued=194242390 : : -: : : : e=easy¬new=yes : : : -: : : : : 7 : : -: : : : : 6 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-11:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-03:201 : : -: : : : 013-07-15,2013-07-01:2013-07-14,2013-07-05:2013-07-13&stat_type=ca : 3-07-15,2013-07-11:2013-07-15,2013-07-01:2013-07-14&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&target_al : ate&y1=2013&m1=07&d1=03&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : l=1 : : : -: : : : : 83.0 : : -: : : : howCamps&ulogin=nordex-grandit : 57.0&cmd=showCampStat&detail=Yes&types=days&cid=6947313&ulogin=nord : : -: : : : : ex-grandit : : -: : : : : =Yes&types=days&cid=7068093 : : -: : : : : =Yes&types=days&cid=6834748 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2011-10-27:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2011-10-27:201 : : -: : : : 011-10-27,2011-08-31:2011-08-31,2011-08-26:2011-08-26&stat_type=ca : 1-10-27,2011-08-31:2011-08-31,2011-08-26:2011-08-26&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=14&y2=2013&m2=07&d2=14&group=day&target_al : ate&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : l=1 : : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 38 : 07.0&authredirlevel=1373876446.0 : : -: : : : : 134782&cmd=showCamp&fixated=150055634 : : -: : : : : dirlevel=1364366494.0&cmd=showCamp&cid=6106774 : : -: : : : : =Yes&types=days&cid=7072622 : : -: : : : : owClients : : -: : : : : 113103&cmd=showCamp : : -: : : : : 2 : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : : 5 : : -: : : : : 05.6&authredirlevel=1372014905.5&authredirlevel=1372014905.4&authre : : -: : : : : dirlevel=1372014905.3&authredirlevel=1372014905.2&authredirlevel=13 : : -: : : : : 72014905.1&authredirlevel=1372014905.0&authredirlevel=1370480737.6& : : -: : : : : authredirlevel=1370480737.5&authredirlevel=1370480737.4&authredirle : : -: : : : : vel=1370480737.3&authredirlevel=1370480737.2&authredirlevel=1370480 : : -: : : : : 737.1&authredirlevel=1370480737.0&cmd=showCampStat&stat_type=campda : : -: : : : : te&target_0=1&target_1=1&target_all=1 : : -: : : : : ulogin= : : -: : : : : 5 : : -: : : : howCamp&cid=6980076 : owCampStat&cid=6980076&detail=Yes&types=days : : -: : : : &ulogin= : ulogin= : : -: : : : howClients : owCamps&ulogin=K-PV-beeline : : -: : : : l=Yes&types=days&cid=88689 : : : -: : : : 32 : : : -: : : : 15 : owCampStat&cid=5991615&detail=Yes&types=days : : -: : : : howCamp&cid=6980076 : owCamp&cid=6980076 : : -: : : : : =Yes&types=days&cid=5871623 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 8 : : -: : : : : owCampStat&cid=4917291&detail=Yes&types=days : : -: : : : : 2 : : -: : : : : 90.0 : : -: : : : : 8 : : -: : : : : owCamps&ulogin=domusmebel-ru : : -: : : : : 0 : : -: : : : : =Yes&types=days&cid=4735372 : : -: : : : : 9 : : -: : : : 49 : : : -: : : : : 5 : : -: : : : 08 : : : -: : : : howCampStat&cid=3353427&offline_stat=0&online_stat=0&stat_periods= : d2=15&target_0=0&stat_periods=2013-07-01:2013-07-06,2013-06-27:2013 : : -: : : : 2013-07-01:2013-07-06,2013-06-27:2013-07-05,2013-06-01:2013-06-30& : -07-05,2013-06-01:2013-06-30&m2=07&stat_type=custom&onpage=100&grou : : -: : : : stat_type=custom&sort=&reverse=ARRAY(0x1639e200)&group_by=tag&filt : p_by=tag, banner, page, phrase&d1=15&target_1=0&cmd=showCampStat&ta : : -: : : : er_tag=&group_by=banner&filter_banner=&group_by=page&filter_page=& : rget_all=1&online_stat=0&m1=07&cid=3353427&goals=0&offline_stat=0&y : : -: : : : filter_page_target=&group_by=phrase&filter_phrase=&goals=0&y1=2013 : 2=13&sort=clicks&reverse=1 : : -: : : : &m1=07&d1=15&y2=2013&m2=07&d2=15&onpage=100 : : : -: : : : : owCamps : : -: : : : : 2 : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 8 : : -: : : : 44 : owCampStat&cid=4154744&detail=Yes&types=days : : -: : : : 01 : : : -: : : : site_promotion_search : : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 02.0 : : -: : : : : 7 : : -: : : : howCamp&cid=6522990&tab=off : owCamp&cid=6522990 : : -: : : : : 9 : : -: : : : : 8 : : -: : : : : 9 : : -: : : : : 5 : : -: : : : : 4 : : -: : : : 813.0&cmd=showCamp&cid=5210661 : : : -: : : : : 63.0&cmd=showCamp&cid=5830159 : : -: : : : : =Yes&types=days&cid=6993362 : : -: : : : : vancedForecast : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : 7090667&cmd=showCamp : owCampStat&cid=7090667&detail=Yes&types=days : : -: : : : 77&ulogin=svk-direct : : : -: : : : : 3 : : -: : : : : 1 : : -: : : : howCampStat&cid=5975585&detail=Yes&types=days : 5 : : -: : : : : 8 : : -: : : : : 1 : : -: : : : : 41.0&authredirlevel=1373879619.0&cmd=showCamp&cid=5356346 : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-03-01:2 : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-03-01:201 : : -: : : : 013-07-15,2013-06-01:2013-06-14,2013-06-01:2013-07-15&stat_type=ca : 3-07-15,2013-06-01:2013-06-14,2013-06-01:2013-07-15&stat_type=campd : : -: : : : mpdate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=month&target_ : ate&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : all=1 : : : -: : : : : owCamps&ulogin=infoprom51 : : -: : : : : 64.0&cmd=showCamp&cid=5028648 : : -: : : : 64 : : : -: : : : : 6 : : -: : : : : 34.0&cmd=ForecastByWords : : -: : : : : 4 : : -: : : : ts : : : -: : : : : 2 : : -: : : : ncy : : : -: : : : : 8 : : -: : : : : 55.0&cmd=showCampStat&detail=Yes&types=days&cid=6849113 : : -: : : : howCamps&ulogin=aplarru : 9&ulogin=aplarru : : -: : : : : =Yes&types=days&cid=5039376 : : -: : : : : owCamp&cid=4748947&ulogin=cont5nn#71488518 : : -: : : : _camps=0&cmd=showSubClientCamps&authredirlevel=1373873308.0&settin : dirlevel=1373880608.0&media_camps=0&cmd=showSubClientCamps&authredi : : -: : : : gs_sorted=&sort=total&reverse=1 : rlevel=1373873308.0&settings_sorted=&sort=total&reverse=0 : : -: : : : howCamp&cid=4581573&tab=decline : owCamp&cid=4581573&tab=active : : -: : : : : 9 : : -: : : : howClients : vancedForecast : : -: : : : : 983763&cmd=showCamp&unglued=194305392 : : -: : : : : 9 : : -: : : : : 5 : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=7030009 : : -: : : : 0 : : : -: : : : : 8 : : -: : : : : 9 : : -: : : : : 65.0&cmd=showCamp&cid=7135509 : : -: : : : 642.0 : ype=campdate : : -: : : : : 5 : : -: : : : 35 : 5 : : -: : : : : dirlevel=1373877895.0&cmd=showCamp&cid=3055906 : : -: : : : : 6 : : -: : : : 099442&cid=7100140&cid=7100144&cid=7100148&cid=7100150&cid=7132172 : owCampStat&cid=7099442&cid=7100140&cid=7100144&cid=7100148&cid=7100 : : -: : : : &cid=7132174&cid=7132175&cid=7132177&cid=7132179&cid=7132632&cid=7 : 150&cid=7132172&cid=7132174&cid=7132175&cid=7132177&cid=7132179&cid : : -: : : : 132838&cid=7132888&cid=7133059&cid=7133217&stat_type=campdate&csrf : =7132632&cid=7132838&cid=7132888&cid=7133059&cid=7133217&offline_st : : -: : : : _token=aENmaxzslTI5oMqD : at=0&online_stat=0&stat_periods=2013-07-01:2013-07-13,2013-07-01:20 : : -: : : : : 13-07-12,2013-06-14:2013-07-11&stat_type=campdate&y1=2013&m1=07&d1= : : -: : : : : 15&y2=2013&m2=07&d2=15&group=day&target_all=1 : : -: : : : 35 : 63.0&authredirlevel=1373873632.0 : : -: : : : : 1 : : -: : : : 73 : : : -: : : : howCampStat&cid=6981382&offline_stat=0&online_stat=1&stat_periods= : owCampStat&cid=6981382&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : 2013-06-10:2013-06-25,2013-06-10:2013-06-24,&detail=yes&page=&y1=2 : 13-06-10:2013-06-25,2013-06-10:2013-06-24,&detail=yes&page=&y1=2013 : : -: : : : 013&m1=07&d1=05&y2=2013&m2=07&d2=12&group=day&goals=0&target_all=y : &m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : es : : : -: : : : : =Yes&types=days&cid=6621650 : : -: : : : gSearch&who=camps&cid=&camp_login=&camp_name=атлант : 4&ulogin=atlant-b2bnew : : -: : : : : 0 : : -: : : : : owCamps&ulogin=domusmebel-ru : : -: : : : : 3 : : -: : : : : owCamp&cid=3601040 : : -: : : : : =Yes&types=days&cid=6858643 : : -: : : : : =Yes&types=days&cid=7109462 : : -: : : : : =Yes&types=days&cid=7085888 : : -: : : : 25 : : : -: : : : : owCamps : : -: : : : d=18770681&resp_desc=success : owCamps : : -: : : : howCamp&cid=6980076 : owCampStat&cid=6980076&detail=Yes&types=days : : -: : : : : 5 : : -: : : : : 4 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=5129796 : : -: : : : : 7 : : -: : : : howCampStat&cid=7104084&detail=Yes&types=days : 104084&cmd=showCamp : : -: : : : : 6 : : -: : : : : 2 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=3427083 : : : -: : : : : =Yes&types=days&cid=7060114 : : -: : : : l=Yes&types=days&cid=6833696 : owCampStat&cid=6833696&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-20:2013-06-21,&detail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m : : -: : : : : 2=07&d2=15&group=day&goals=0&target_all=yes : : -: : : : : 137972&cmd=showCamp&unglued=194336987 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : &mediaType=text&firstbanner=std : : -: : : : 137830&cmd=orderCamp&agree=yes : : : -: : : : : 9 : : -: : : : 09 : : : -: : : : : 131075&cmd=showCamp : : -: : : : : 2 : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6960697 : : -: : : : 7129022&cmd=showCamp : owCampStat&cid=7129022&detail=Yes&types=days : : -: : : : 43 : : : -: : : : : 8 : : -: : : : 17 : owCamp&cid=7082517 : : -: : : : : 6 : : -: : : : 31&ulogin=kuncevo-ru : : : -: : : : 394451&phrasedate=Yes&target_all=1 : owCampStat&cid=4394451&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 12-01-01:2012-11-21,2012-01-01:2012-07-31,2012-03-23:2012-03-23&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 1 : : -: : : : howCamps : owCamps : : -: : : : : 1 : : -: : : : : 6 : : -: : : : gSearch&who=camps&cid=&camp_login=&camp_name=дизв : 9&ulogin=dizvitr-b2b : : -: : : : : 7 : : -: : : : : 9 : : -: : : : : campdate : : -: : : : : 0 : : -: : : : : 6 : : -: : : : : 6 : : -: : : : howCamps&ulogin=oknabuduschego : =Yes&types=days&cid=5506368&ulogin=oknabuduschego : : -: : : : howClients&sort=total&reverse=0 : owCamps&ulogin=Rusokon2007 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=4879898 : : -: : : : 1&mediaType=text : : : -: : : : 85 : : : -: : : : howCamps&ulogin=visacity08 : 5&ulogin=visacity08 : : -: : : : 56 : : : -: : : : : owCamps&ulogin=dc12a : : -: : : : 59 : : : -: : : : howCamps&ulogin=ul-smilestar-su : =ul-smilestar-su#list : : -: : : : howCampStat&cid=7053221&detail=Yes&types=days : owCampStat&cid=7053221&offline_stat=0&online_stat=0&stat_periods=&d : : -: : : : : etail=yes&page=&y1=2013&m1=07&d1=08&y2=2013&m2=07&d2=15&group=day&g : : -: : : : : oals=0&target_all=yes&show_banners_stat=1 : : -: : : : : =Yes&types=days&cid=7133318 : : -: : : : : 5 : : -: : : : 25 : : : -: : : : howCamp&tab=all&cid=7126549&ulogin=transkontinental2014#194270034 : : : -: : : : : 2 : : -: : : : 02 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 07 : : : -: : : : : 1 : : -: : : : : 9 : : -: : : : howCamp&cid=7078898&tab=off : owCamp&cid=7078898 : : -: : : : 01 : : : -: : : : : 4 : : -: : : : : 35.0 : : -: : : : : owCampStat&cid=2886087&detail=Yes&types=days : : -: : : : : 8 : : -: : : : type=campdate : dirlevel=1373865822.0&cmd=showCampStat&offline_stat=0&online_stat=0 : : -: : : : : &stat_periods=2013-07-04:2013-07-05,2013-06-10:2013-06-13,2013-05-2 : : -: : : : : 1:2013-05-31&stat_type=campdate&y1=2013&m1=07&d1=15&y2=2013&m2=07&d : : -: : : : : 2=15&group=day&target_all=1 : : -: : : : : 0 : : -: : : : : owCamp&cid=6354573#194189392 : : -: : : : : owCamps : : -: : : : 25 : : : -: : : : : 8 : : -: : : : : 04.0&cmd=showCamp&cid=7129428 : : -: : : : howCamp&cid=7130323 : ids=193775525&cid=7130323 : : -: : : : howCamps&ulogin=richton-studio : 7&ulogin=richton-studio : : -: : : : : owCampStat&cid=7034406&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-07-11:2013-07-11,2013-07-12:2013-07-12,2013-07-13:2013-07-13&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : : 0 : : -: : : : howCamp&tab=decline&cid=6886514 : owCamp&cid=6886514&tab=active : : -: : : : : 33.0 : : -: : : : : 60.0 : : -: : : : : 5 : : -: : : : : dirlevel=1373866480.0&cmd=showCampStat&stat_type=by_agency_clients : : -: : : : : 9 : : -: : : : : 463697&cmd=showCamp : : -: : : : 28 : : : -: : : : : 6 : : -: : : : : 8 : : -: : : : : 4 : : -: : : : 60 : : : -: : : : : 5 : : -: : : : : 7 : : -: : : : : 1 : : -: : : : howCampStat&cid=6456112&offline_stat=0&online_stat=0&stat_periods= : owCampStat&cid=6456112&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : 2013-07-10:2013-07-10,2013-07-09:2013-07-09,2013-07-08:2013-07-08& : 13-07-10:2013-07-10,2013-07-09:2013-07-09,2013-07-08:2013-07-08&phr : : -: : : : phrasedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group : asedate=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day : : -: : : : =day&goals=0&target_all=yes : &goals=0&target_all=yes : : -: : : : : 7 : : -: : : : : 3 : : -: : : : howCampStat&cid=4121349&detail=Yes&types=days : owCampStat&cid=4121349&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-07-08:2013-07-09,2013-06-01:2013-06-06,2013-04-01:2013-06-03&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=15&y2=2013&m2=07&d2=15&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : : owClients : : -: : : : 15 : : : -: : : : l=Yes&types=days&cid=7103094 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7103094&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 4 : : -: : : : 84 : : : -: : : : 89 : : : -: : : : 1&mediaType=text : : : -: : : : : owClients : : -: : : : : 2 : : -: : : : : 2 : : -: : : : 86 : : : -: : : : : 4 : : -: : : : 12 : : : -: : : : : 6 : : -: : : : : owCamps&ulogin=dc12a : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 9 : : -: : : : l=Yes&types=days&cid=141133 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=141133&sho : : -: : : : : w_banners_stat=1 : : -: : : : : 108927&cmd=showCamp : : -: : : : howCamp&cid=6886521&tab=off : owCamp&cid=6886521 : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=7133822 : : -: : : : : owCamps&ulogin=mitsubishi-galant2013 : : -: : : : ditCamp&cid=7088102 : owCamp&cid=7088102 : : -: : : : l=Yes&types=days&cid=5033617 : : : -: : : : : 2 : : -: : : : 56 : : : -: : : : : =Yes&types=days&cid=5056684 : : -: : : : 25 : : : -: : : : : owCamps : : -: : : : l=Yes&types=days&cid=6479409 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=6479409&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 9 : : -: : : : : 85.0 : : -: : : : : =Yes&types=days&cid=7133333 : : -: : : : : owCampStat&cid=3618668&detail=Yes&types=days : : -: : : : 74 : owCamp&cid=4275274&tag=303211 : : -: : : : : =Yes&types=days&cid=3298907 : : -: : : : : 9 : : -: : : : : 1 : : -: : : : howCamp&cid=5931713&tab=wait : owCamp&cid=5931713&tab=active : : -: : : : : &mediaType=text : : -: : : : 22 : : : -: : : : 002.0 : 7 : : -: : : : : e_stat=0&online_stat=0&stat_type=campdate&group=day&target_0=1&targ : : -: : : : : et_1=1 : : -: : : : : 4 : : -: : : : mportCampSuccess&cid=4202002&make_new_camp=0 : owCamp&cid=4202002 : : -: : : : : 5 : : -: : : : 87 : : : -: : : : howCamps : owCamps : : -: : : : 99 : : : -: : : : : 3 : : -: : : : 90 : 0 : : -: : : : edirlevel=1373611946.0&authredirlevel=1372140313.0&authredirlevel= : : : -: : : : 1371447906.0&authredirlevel=1368426548.0&authredirlevel=1367818443 : : : -: : : : .0&cmd=showCamps&ulogin=Newdesign-Larus : : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : 97 : : : -: : : : : 9 : : -: : : : howCamps&ulogin=ogurcova-mkt : =Yes&types=days&cid=4139609&ulogin=ogurcova-mkt : : -: : : : : 2 : : -: : : : : 2 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : : 136655&cmd=showCamp : : -: : : : : 8 : : -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -1000 tuples (4.9s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (2.8s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (50.916ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (41.480ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' A ND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (3.0s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' A ND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (45.935ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' A ND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (47.277ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -+---+-----------+ -+===+===========+ -+---+-----------+ -0 tuples (2.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -+---+-----------+ -+===+===========+ -+---+-----------+ -0 tuples (49.119ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_100m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -+---+-----------+ -+===+===========+ -+---+-----------+ -0 tuples (48.280ms) -sql>\q -stop time: Вт. сент. 17 18:29:18 MSK 2013 diff --git a/benchmark/monetdb/log/log_10m b/benchmark/monetdb/log/log_10m deleted file mode 100644 index 70baf28d826..00000000000 --- a/benchmark/monetdb/log/log_10m +++ /dev/null @@ -1,2250 +0,0 @@ -start time: Пн. сент. 2 20:45:29 MSK 2013 - -times: 1 -query: SELECT count(*) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m; -+----------+ -+==========+ -+----------+ -1 tuple (47.434ms) -sql>\q -monetdbd is running - -times: 2 -query: SELECT count(*) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m; -+----------+ -+==========+ -+----------+ -1 tuple (3.404ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT count(*) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m; -+----------+ -+==========+ -+----------+ -1 tuple (3.524ms) -sql>\q -monetdbd is running - -times: 1 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID not like 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE AdvEngineID not like 0; -+--------+ -+========+ -+--------+ -1 tuple (5.4s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID not like 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE AdvEngineID not like 0; -+--------+ -+========+ -+--------+ -1 tuple (5.3s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID not like 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE AdvEngineID not like 0; -+--------+ -+========+ -+--------+ -1 tuple (5.2s) -sql>\q -monetdbd is running - -times: 1 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+---------+----------+--------------------------+ -+=========+==========+==========================+ -+---------+----------+--------------------------+ -1 tuple (508.823ms) -sql>\q -monetdbd is running - -times: 2 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+---------+----------+--------------------------+ -+=========+==========+==========================+ -+---------+----------+--------------------------+ -1 tuple (24.691ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+---------+----------+--------------------------+ -+=========+==========+==========================+ -+---------+----------+--------------------------+ -1 tuple (23.621ms) -sql>\q -monetdbd is running - -times: 1 -query: SELECT sum(UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_10m; -overflow in calculation. -sql>\q -monetdbd is running - -times: 2 -query: SELECT sum(UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_10m; -overflow in calculation. -sql>\q -monetdbd is running - -times: 3 -query: SELECT sum(UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_10m; -overflow in calculation. -sql>\q -monetdbd is running - -times: 1 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (594.911ms) -sql>\q -monetdbd is running - -times: 2 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (560.357ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (548.006ms) -sql>\q -monetdbd is running - -times: 1 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (4.2s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (2.5s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (2.6s) -sql>\q -monetdbd is running - -times: 1 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_10m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (169.017ms) -sql>\q -monetdbd is running - -times: 2 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_10m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (8.401ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_10m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (4.369ms) -sql>\q -monetdbd is running - -times: 1 -query: SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID not like 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID not like 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+-------+ -+=============+=======+ -+-------------+-------+ -9 tuples (5.2s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID not like 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID not like 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+-------+ -+=============+=======+ -+-------------+-------+ -9 tuples (5.3s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID not like 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID not like 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+-------+ -+=============+=======+ -+-------------+-------+ -9 tuples (5.2s) -sql>\q -monetdbd is running --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - - -times: 1 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+==========+========+ -+----------+--------+ -10 tuples (6.2s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+==========+========+ -+----------+--------+ -10 tuples (5.7s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+==========+========+ -+----------+--------+ -10 tuples (5.7s) -sql>\q -monetdbd is running --- агрегация, среднее количество ключей.; - - -times: 1 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -+----------+--------+---------+--------------------------+--------+ -+==========+========+=========+==========================+========+ -+----------+--------+---------+--------------------------+--------+ -10 tuples (8.1s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -+----------+--------+---------+--------------------------+--------+ -+==========+========+=========+==========================+========+ -+----------+--------+---------+--------------------------+--------+ -10 tuples (8.3s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -+----------+--------+---------+--------------------------+--------+ -+==========+========+=========+==========================+========+ -+----------+--------+---------+--------------------------+--------+ -10 tuples (8.3s) -sql>\q -monetdbd is running --- агрегация, среднее количество ключей, несколько агрегатных функций.; - - -times: 1 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel not like '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel not like '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+---------+ -+==================+=========+ -+------------------+---------+ -1 tuple (1.5s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel not like '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel not like '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+---------+ -+==================+=========+ -+------------------+---------+ -1 tuple (1.4s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel not like '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel not like '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+---------+ -+==================+=========+ -+------------------+---------+ -1 tuple (1.4s) -sql>\q -monetdbd is running --- мощная фильтрация по строкам, затем агрегация по строкам.; - - -times: 1 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel not like '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel not like '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -+-------------+------------------+---------+ -+=============+==================+=========+ -+-------------+------------------+---------+ -9 tuples (1m 32s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel not like '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel not like '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -+-------------+------------------+---------+ -+=============+==================+=========+ -+-------------+------------------+---------+ -9 tuples (1m 20s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel not like '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel not like '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -+-------------+------------------+---------+ -+=============+==================+=========+ -+-------------+------------------+---------+ -9 tuples (1m 30s) -sql>\q -monetdbd is running --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - - -times: 1 -query: SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------+---------+ -+==============+=========+ -+--------------+---------+ -1 tuple (421.853ms) -sql>\q -monetdbd is running - -times: 2 -query: SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------+---------+ -+==============+=========+ -+--------------+---------+ -1 tuple (437.020ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------+---------+ -+==============+=========+ -+--------------+---------+ -1 tuple (494.527ms) -sql>\q -monetdbd is running --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - - -times: 1 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+--------------+---------+ -+==============+=========+ -+--------------+---------+ -1 tuple (1.2s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+--------------+---------+ -+==============+=========+ -+--------------+---------+ -1 tuple (1.2s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+--------------+---------+ -+==============+=========+ -+--------------+---------+ -1 tuple (1.2s) -sql>\q -monetdbd is running --- агрегация чуть сложнее.; - - -times: 1 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+----------------+--------------+---------+ -+================+==============+=========+ -+----------------+--------------+---------+ -10 tuples (1.1s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+----------------+--------------+---------+ -+================+==============+=========+ -+----------------+--------------+---------+ -10 tuples (937.924ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+----------------+--------------+---------+ -+================+==============+=========+ -+----------------+--------------+---------+ -10 tuples (939.490ms) -sql>\q -monetdbd is running --- агрегация по числу и строке, большое количество ключей.; - - -times: 1 -query: SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+ -+=====================+======+ -+---------------------+------+ -10 tuples (662.040ms) -sql>\q -monetdbd is running - -times: 2 -query: SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+ -+=====================+======+ -+---------------------+------+ -10 tuples (649.461ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+ -+=====================+======+ -+---------------------+------+ -10 tuples (646.120ms) -sql>\q -monetdbd is running --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+------+ -+=====================+==============+======+ -+---------------------+--------------+------+ -10 tuples (27.5s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+------+ -+=====================+==============+======+ -+---------------------+--------------+------+ -10 tuples (14.0s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+------+ -+=====================+==============+======+ -+---------------------+--------------+------+ -10 tuples (5.7s) -sql>\q -monetdbd is running --- ещё более сложная агрегация.; - - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+---------------------+------------------------------------+------+ -+=====================+====================================+======+ -+---------------------+------------------------------------+------+ -10 tuples (28.0s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+---------------------+------------------------------------+------+ -+=====================+====================================+======+ -+---------------------+------------------------------------+------+ -10 tuples (5.6s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+---------------------+------------------------------------+------+ -+=====================+====================================+======+ -+---------------------+------------------------------------+------+ -10 tuples (23.8s) -sql>\q -monetdbd is running --- то же самое, но без сортировки.; - - -times: 1 -query: SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+------+ -+=====================+======+==============+======+ -+---------------------+------+--------------+------+ -10 tuples (19.3s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+------+ -+=====================+======+==============+======+ -+---------------------+------+--------------+------+ -10 tuples (18.8s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+------+ -+=====================+======+==============+======+ -+---------------------+------+--------------+------+ -10 tuples (22.9s) -sql>\q -monetdbd is running --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - - -times: 1 -query: SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -+--------+ -+========+ -+--------+ -0 tuples (321.425ms) -sql>\q -monetdbd is running - -times: 2 -query: SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -+--------+ -+========+ -+--------+ -0 tuples (0.696ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -+--------+ -+========+ -+--------+ -0 tuples (0.855ms) -sql>\q -monetdbd is running --- мощная фильтрация по столбцу типа UInt64.; - - -times: 1 -query: SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+------+ -+======+ -+------+ -1 tuple (5.9s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+------+ -+======+ -+------+ -1 tuple (44.325ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+------+ -+======+ -+------+ -1 tuple (38.747ms) -sql>\q -monetdbd is running --- фильтрация по поиску подстроки в строке.; - - -times: 1 -query: SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------+----------------------------------------------------------+------+ -+==============+==========================================================+======+ -+--------------+----------------------------------------------------------+------+ -1 tuple (631.284ms) -sql>\q -monetdbd is running - -times: 2 -query: SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------+----------------------------------------------------------+------+ -+==============+==========================================================+======+ -+--------------+----------------------------------------------------------+------+ -1 tuple (607.214ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+--------------+----------------------------------------------------------+------+ -+==============+==========================================================+======+ -+--------------+----------------------------------------------------------+------+ -1 tuple (607.498ms) -sql>\q -monetdbd is running --- вынимаем большие столбцы, фильтрация по строке.; - - -times: 1 -query: SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPh rase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+------+-------------------------+------------------------------------------------------------------------------------------------------------------------+------+------+ -: chph : : : : : -: rase : : : : : -+======+=========================+========================================================================================================================+======+======+ -: : : льский край, КавМинВоды р-н, Кисловодск - IRR.ru : : : -+------+-------------------------+------------------------------------------------------------------------------------------------------------------------+------+------+ -1 tuple (7.1s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPh rase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+------+-------------------------+------------------------------------------------------------------------------------------------------------------------+------+------+ -: chph : : : : : -: rase : : : : : -+======+=========================+========================================================================================================================+======+======+ -: : : льский край, КавМинВоды р-н, Кисловодск - IRR.ru : : : -+------+-------------------------+------------------------------------------------------------------------------------------------------------------------+------+------+ -1 tuple (1.4s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPh rase not like '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+------+-------------------------+------------------------------------------------------------------------------------------------------------------------+------+------+ -: chph : : : : : -: rase : : : : : -+======+=========================+========================================================================================================================+======+======+ -: : : льский край, КавМинВоды р-н, Кисловодск - IRR.ru : : : -+------+-------------------------+------------------------------------------------------------------------------------------------------------------------+------+------+ -1 tuple (1.4s) -sql>\q -monetdbd is running --- чуть больше столбцы.; - - -times: 1 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------+--------------+------+---------------+------------+------+------------+------+---------------------+------+------+------+----------------+ -: : enab : : even : : : teri : : onid : : terc : : agen : :> -: : le : : t : : : d : : : : lass : : t : :> -+=====================+======+==============+======+===============+============+======+============+======+=====================+======+======+======+================+ -: : : : : :58:04.000000 : : : : : : : : : .yandex.ru/reg : -: : : : : : : : : : : : : : istered/main.p : -: : : : : : : : : : : : : : l?cmd=newCam...> -: : : : : :58:13.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :00:43.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :01:30.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :01:57.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :02:21.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :02:29.000000 : : : : : : : : : .yandex.ru/reg : -: : : : : : : : : : : : : : istered/main.p : -: : : : : : : : : : : : : : l?from=metrika : -: : : : : :02:37.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :04:10.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :04:45.000000 : : : : : : : : : .yandex.ru/ : -+---------------------+------+--------------+------+---------------+------------+------+------------+------+---------------------+------+------+------+----------------+ -10 tuples (45.0s) !92 columns dropped, 1 field truncated! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q -monetdbd is running - -times: 2 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------+--------------+------+---------------+------------+------+------------+------+---------------------+------+------+------+----------------+ -: : enab : : even : : : teri : : onid : : terc : : agen : :> -: : le : : t : : : d : : : : lass : : t : :> -+=====================+======+==============+======+===============+============+======+============+======+=====================+======+======+======+================+ -: : : : : :58:04.000000 : : : : : : : : : .yandex.ru/reg : -: : : : : : : : : : : : : : istered/main.p : -: : : : : : : : : : : : : : l?cmd=newCam...> -: : : : : :58:13.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :00:43.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :01:30.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :01:57.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :02:21.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :02:29.000000 : : : : : : : : : .yandex.ru/reg : -: : : : : : : : : : : : : : istered/main.p : -: : : : : : : : : : : : : : l?from=metrika : -: : : : : :02:37.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :04:10.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :04:45.000000 : : : : : : : : : .yandex.ru/ : -+---------------------+------+--------------+------+---------------+------------+------+------------+------+---------------------+------+------+------+----------------+ -10 tuples (501.493ms) !92 columns dropped, 1 field truncated! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q -monetdbd is running - -times: 3 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------+--------------+------+---------------+------------+------+------------+------+---------------------+------+------+------+----------------+ -: : enab : : even : : : teri : : onid : : terc : : agen : :> -: : le : : t : : : d : : : : lass : : t : :> -+=====================+======+==============+======+===============+============+======+============+======+=====================+======+======+======+================+ -: : : : : :58:04.000000 : : : : : : : : : .yandex.ru/reg : -: : : : : : : : : : : : : : istered/main.p : -: : : : : : : : : : : : : : l?cmd=newCam...> -: : : : : :58:13.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :00:43.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :01:30.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :01:57.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :02:21.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :02:29.000000 : : : : : : : : : .yandex.ru/reg : -: : : : : : : : : : : : : : istered/main.p : -: : : : : : : : : : : : : : l?from=metrika : -: : : : : :02:37.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :04:10.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :04:45.000000 : : : : : : : : : .yandex.ru/ : -+---------------------+------+--------------+------+---------------+------------+------+------------+------+---------------------+------+------+------+----------------+ -10 tuples (504.824ms) !92 columns dropped, 1 field truncated! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q -monetdbd is running --- плохой запрос - вынимаем все столбцы.; - - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY EventTime LIMIT 10; -+--------------+ -+==============+ -+--------------+ -10 tuples (519.376ms) -sql>\q -monetdbd is running - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY EventTime LIMIT 10; -+--------------+ -+==============+ -+--------------+ -10 tuples (511.995ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY EventTime LIMIT 10; -+--------------+ -+==============+ -+--------------+ -10 tuples (502.501ms) -sql>\q -monetdbd is running --- большая сортировка.; - - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY SearchPhrase LIMIT 10; -+--------------+ -+==============+ -+--------------+ -10 tuples (463.535ms) -sql>\q -monetdbd is running - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY SearchPhrase LIMIT 10; -+--------------+ -+==============+ -+--------------+ -10 tuples (450.970ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY SearchPhrase LIMIT 10; -+--------------+ -+==============+ -+--------------+ -10 tuples (446.560ms) -sql>\q -monetdbd is running --- большая сортировка по строкам.; - - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+--------------+ -+==============+ -+--------------+ -10 tuples (520.424ms) -sql>\q -monetdbd is running - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+--------------+ -+==============+ -+--------------+ -10 tuples (487.075ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase not like '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+--------------+ -+==============+ -+--------------+ -10 tuples (485.532ms) -sql>\q -monetdbd is running --- большая сортировка по кортежу.; - - -times: 1 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL not like '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL not like '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -+-----------+---+----+ -+===========+===+====+ -+-----------+---+----+ -0 tuples (846.848ms) -sql>\q -monetdbd is running - -times: 2 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL not like '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL not like '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -+-----------+---+----+ -+===========+===+====+ -+-----------+---+----+ -0 tuples (243.749ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL not like '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL not like '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -+-----------+---+----+ -+===========+===+====+ -+-----------+---+----+ -0 tuples (241.169ms) -sql>\q -monetdbd is running --- считаем средние длины URL для крупных счётчиков.; - - -times: 1 -query: SELECT SUBSTRING(SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, GREATEST(0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1)) AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer not like '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING(SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, GREATEST(0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1)) AS k, avg (length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer not like '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -SELECT: no such binary operator 'greatest(bigint,bigint)' -sql>\q -monetdbd is running - -times: 2 -query: SELECT SUBSTRING(SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, GREATEST(0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1)) AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer not like '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING(SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, GREATEST(0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1)) AS k, avg (length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer not like '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -SELECT: no such binary operator 'greatest(bigint,bigint)' -sql>\q -monetdbd is running - -times: 3 -query: SELECT SUBSTRING(SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, GREATEST(0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1)) AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer not like '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING(SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, GREATEST(0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1)) AS k, avg (length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer not like '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -SELECT: no such binary operator 'greatest(bigint,bigint)' -sql>\q -monetdbd is running --- то же самое, но с разбивкой по доменам.; - - -times: 1 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(Re solutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWid th + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18 ), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum( ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(Resolut ionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidt h + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43) , sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(R esolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(Resoluti onWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(Re solutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(Resolutio nWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+ -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -1 tuple (1.0s) !78 columns dropped! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q -monetdbd is running - -times: 2 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(Re solutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWid th + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18 ), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum( ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(Resolut ionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidt h + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43) , sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(R esolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(Resoluti onWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(Re solutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(Resolutio nWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+ -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -1 tuple (800.552ms) !78 columns dropped! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q -monetdbd is running - -times: 3 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(Re solutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWid th + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18 ), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum( ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(Resolut ionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidt h + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43) , sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(R esolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(Resoluti onWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(Re solutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(Resolutio nWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+ -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -1 tuple (819.192ms) !78 columns dropped! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q -monetdbd is running --- много тупых агрегатных функций.; - - -times: 1 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, ClientIP ORD ER BY count(*) DESC LIMIT 10; -+----------------+------------+------+------+--------------------------+ -+================+============+======+======+==========================+ -+----------------+------------+------+------+--------------------------+ -10 tuples (36.9s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, ClientIP ORD ER BY count(*) DESC LIMIT 10; -+----------------+------------+------+------+--------------------------+ -+================+============+======+======+==========================+ -+----------------+------------+------+------+--------------------------+ -10 tuples (36.2s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY SearchEngineID, ClientIP ORD ER BY count(*) DESC LIMIT 10; -+----------------+------------+------+------+--------------------------+ -+================+============+======+======+==========================+ -+----------------+------------+------+------+--------------------------+ -10 tuples (36.1s) -sql>\q -monetdbd is running --- сложная агрегация, для больших таблиц может не хватить оперативки.; - - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (2.9s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (2.3s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase not like '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (2.3s) -sql>\q -monetdbd is running --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (1.2s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (1.2s) -sql>\q -monetdbd is running - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (1.2s) -sql>\q -monetdbd is running --- то же самое, но ещё и без фильтрации.; - - -times: 1 -query: SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -Connection terminated -monetdbd is running - -times: 2 -query: SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -unsupported hash algorithms: gorithms: gorit -monetdbd is running - -times: 3 -query: SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -unsupported hash algorithms: gorithms: g -monetdbd is running --- агрегация по URL.; - - -times: 1 -query: SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -unsupported hash algorithms: gorithms: gorit -monetdbd is running - -times: 2 -query: SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -unsupported hash algorithms: gorithms: gori -monetdbd is running - -times: 3 -query: SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -monetdbd: internal error while starting mserver, please refer to the logs - -monetdbd is running --- агрегация по URL и числу.; - - -times: 1 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_10m group by clientip, clientip -" -sql>\q -monetdbd is running - -times: 2 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_10m group by clientip, clientip -" -sql>\q -monetdbd is running - -times: 3 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_10m group by clientip, clientip -" -sql>\q -monetdbd is running - -times: 1 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running - -times: 2 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running - -times: 3 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL not like '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT R efresh AND URL not like '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+------+-----------+ -+======+===========+ -+------+-----------+ -1 tuple (6.4s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL not like '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT R efresh AND URL not like '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+------+-----------+ -+======+===========+ -+------+-----------+ -1 tuple (309.846ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL not like '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT R efresh AND URL not like '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+------+-----------+ -+======+===========+ -+------+-----------+ -1 tuple (300.833ms) -sql>\q -monetdbd is running - -times: 1 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title not like '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refr esh AND Title not like '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+-------+-----------+ -+=======+===========+ -+-------+-----------+ -1 tuple (6.8s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title not like '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refr esh AND Title not like '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+-------+-----------+ -+=======+===========+ -+-------+-----------+ -1 tuple (310.018ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title not like '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refr esh AND Title not like '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+-------+-----------+ -+=======+===========+ -+-------+-----------+ -1 tuple (299.446ms) -sql>\q -monetdbd is running - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT I sDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+---------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ -: : iews : -+===============================================================================================================================================================+=======+ -: c_aft_spring-offer2013:cpc:yandex-direct : : -: =true : : -: hotelClass=all&meal=all&priceFrom=10000&priceTo=300000 : : -: 475 : : -: q2-2013:cpc:yandex-direct:apec2013 : : -: ks:cpc:yandex-direct:tyagach : : -: 0_pkw_stock:cpc:yandex-direct:e200 : : -: {keyword} : : -: 68ae4aada9d8779c3d9db2a299e : : -: apartment&search_form[currency]=RUB&search_form[price_type]=total&search_form[rooms][]=4&search_form[only_with_photo]=true : : -: BJ_PRICE_MIN=&OBJ_PRICE_MAX=§or=&ring=&SORT_BY=0utm_source=Yandex_CPC : : -: hotelClass=all&meal=all&priceFrom=10000&priceTo=300000 : : -: n=true : : -: -01:2013-07-01&m2=07&stat_type=custom&group_by=banner, page, phrase&onpage=100&d1=02&target_1=0&cmd=showCampStat&filter_page_target=context&reverse=1&target_ : : -: all=1&sort=aconv&m1=07&online_stat=1&cid=4828884&goals=0&y2=13&offline_stat=0 : : -: yagach : : -: tml?csref=_030_pkw_stock:cpc:yandex-direct:a-class : : -: _q2-2013:cpc:yandex-direct:edition-c : : -: =day&onpage=10000&group_by=page, phrase, retargeting, position, image, date&d1=07&target_1=0&cmd=showCampStat&reverse=ARRAY(0x14a25080)&target_all=1&m1=07&on : : -: line_stat=0&cid=7088688&goals=0&offline_stat=0&y2=13 : : -: ion=Прочитайте истории успеха клие : : -: ddphrases}&keyword={keyword} : : -: operty_type]=apartment&advert_search[rooms][]=4&advert_search[price_type]=total : : -: categoryID=560&search_name=Huter&search_price_from=0&search_price_to=38625 : : -: 475 : : -: pe}&position={position}&campid=komplekt_yandex_poisk : : -: ={keyword} : : -: =28 : : -: MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=§or=&ring=&clear_cache=Y&SORT_BY=0utm_source=Yandex_CPC : : -: 06-01:2013-06-27&group=day&d1=01&cmd=showCampStat&ulogin=elama-13666725&target_all=yes&phrasedate=yes&online_stat=0&m1=06&cid=6887844&goals=0&offline_stat=0& : : -: y2=13 : : -: n} : : -: n=true : : -: n=true : : -: 2013-06-27&m2=06&group=month&d1=01&cmd=showCampStat&ulogin=beritedengi-2013&target_all=yes&phrasedate=yes&m1=06&online_stat=0&cid=6870145&goals=0&offline_sta : : -: t=0&y2=13 : : -: ice%3D0;35000%26price_from%3D0%26price_to%3D35000%26FILTER[956][]%3D1%26data_form%3DY : : -: ARE_MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=&OBJ_PERIOD_MIN=1&OBJ_PERIOD_MAX=12&SORT_BY=0&names=2574 : : -: 10000&search_subcats=1&= Найти &action=search : : -: _MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=&OBJ_DISTANCE_MIN=&OBJ_DISTANCE_MAX=&SORT_BY=0&names=3158,3669 : : -: 2013-07-01&m2=06&group=week&d1=24&cmd=showCampStat&ulogin=sumki-rekurs&target_all=yes&phrasedate=yes&m1=06&online_stat=0&cid=4054100&goals=0&offline_stat=0&y : : -: 2=13 : : -: d=dirOuyaConsole : : -: 2013-06-24&m2=07&group=year&d1=23&cmd=showCampStat&target_all=yes&phrasedate=yes&online_stat=0&m1=11&cid=4272756&goals=0&offline_stat=0&y2=13 : : -+---------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ -1000 tuples (1.9s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT I sDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+---------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ -: : iews : -+===============================================================================================================================================================+=======+ -: c_aft_spring-offer2013:cpc:yandex-direct : : -: =true : : -: hotelClass=all&meal=all&priceFrom=10000&priceTo=300000 : : -: 475 : : -: q2-2013:cpc:yandex-direct:apec2013 : : -: ks:cpc:yandex-direct:tyagach : : -: 0_pkw_stock:cpc:yandex-direct:e200 : : -: {keyword} : : -: 68ae4aada9d8779c3d9db2a299e : : -: apartment&search_form[currency]=RUB&search_form[price_type]=total&search_form[rooms][]=4&search_form[only_with_photo]=true : : -: BJ_PRICE_MIN=&OBJ_PRICE_MAX=§or=&ring=&SORT_BY=0utm_source=Yandex_CPC : : -: hotelClass=all&meal=all&priceFrom=10000&priceTo=300000 : : -: n=true : : -: -01:2013-07-01&m2=07&stat_type=custom&group_by=banner, page, phrase&onpage=100&d1=02&target_1=0&cmd=showCampStat&filter_page_target=context&reverse=1&target_ : : -: all=1&sort=aconv&m1=07&online_stat=1&cid=4828884&goals=0&y2=13&offline_stat=0 : : -: yagach : : -: tml?csref=_030_pkw_stock:cpc:yandex-direct:a-class : : -: _q2-2013:cpc:yandex-direct:edition-c : : -: =day&onpage=10000&group_by=page, phrase, retargeting, position, image, date&d1=07&target_1=0&cmd=showCampStat&reverse=ARRAY(0x14a25080)&target_all=1&m1=07&on : : -: line_stat=0&cid=7088688&goals=0&offline_stat=0&y2=13 : : -: ion=Прочитайте истории успеха клие : : -: ddphrases}&keyword={keyword} : : -: operty_type]=apartment&advert_search[rooms][]=4&advert_search[price_type]=total : : -: categoryID=560&search_name=Huter&search_price_from=0&search_price_to=38625 : : -: 475 : : -: pe}&position={position}&campid=komplekt_yandex_poisk : : -: ={keyword} : : -: =28 : : -: MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=§or=&ring=&clear_cache=Y&SORT_BY=0utm_source=Yandex_CPC : : -: 06-01:2013-06-27&group=day&d1=01&cmd=showCampStat&ulogin=elama-13666725&target_all=yes&phrasedate=yes&online_stat=0&m1=06&cid=6887844&goals=0&offline_stat=0& : : -: y2=13 : : -: n} : : -: n=true : : -: n=true : : -: 2013-06-27&m2=06&group=month&d1=01&cmd=showCampStat&ulogin=beritedengi-2013&target_all=yes&phrasedate=yes&m1=06&online_stat=0&cid=6870145&goals=0&offline_sta : : -: t=0&y2=13 : : -: ice%3D0;35000%26price_from%3D0%26price_to%3D35000%26FILTER[956][]%3D1%26data_form%3DY : : -: ARE_MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=&OBJ_PERIOD_MIN=1&OBJ_PERIOD_MAX=12&SORT_BY=0&names=2574 : : -: 10000&search_subcats=1&= Найти &action=search : : -: _MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=&OBJ_DISTANCE_MIN=&OBJ_DISTANCE_MAX=&SORT_BY=0&names=3158,3669 : : -: 2013-07-01&m2=06&group=week&d1=24&cmd=showCampStat&ulogin=sumki-rekurs&target_all=yes&phrasedate=yes&m1=06&online_stat=0&cid=4054100&goals=0&offline_stat=0&y : : -: 2=13 : : -: d=dirOuyaConsole : : -: 2013-06-24&m2=07&group=year&d1=23&cmd=showCampStat&target_all=yes&phrasedate=yes&online_stat=0&m1=11&cid=4272756&goals=0&offline_stat=0&y2=13 : : -+---------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ -1000 tuples (361.203ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT I sDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+---------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ -: : iews : -+===============================================================================================================================================================+=======+ -: c_aft_spring-offer2013:cpc:yandex-direct : : -: =true : : -: hotelClass=all&meal=all&priceFrom=10000&priceTo=300000 : : -: 475 : : -: q2-2013:cpc:yandex-direct:apec2013 : : -: ks:cpc:yandex-direct:tyagach : : -: 0_pkw_stock:cpc:yandex-direct:e200 : : -: {keyword} : : -: 68ae4aada9d8779c3d9db2a299e : : -: apartment&search_form[currency]=RUB&search_form[price_type]=total&search_form[rooms][]=4&search_form[only_with_photo]=true : : -: BJ_PRICE_MIN=&OBJ_PRICE_MAX=§or=&ring=&SORT_BY=0utm_source=Yandex_CPC : : -: hotelClass=all&meal=all&priceFrom=10000&priceTo=300000 : : -: n=true : : -: -01:2013-07-01&m2=07&stat_type=custom&group_by=banner, page, phrase&onpage=100&d1=02&target_1=0&cmd=showCampStat&filter_page_target=context&reverse=1&target_ : : -: all=1&sort=aconv&m1=07&online_stat=1&cid=4828884&goals=0&y2=13&offline_stat=0 : : -: yagach : : -: tml?csref=_030_pkw_stock:cpc:yandex-direct:a-class : : -: _q2-2013:cpc:yandex-direct:edition-c : : -: =day&onpage=10000&group_by=page, phrase, retargeting, position, image, date&d1=07&target_1=0&cmd=showCampStat&reverse=ARRAY(0x14a25080)&target_all=1&m1=07&on : : -: line_stat=0&cid=7088688&goals=0&offline_stat=0&y2=13 : : -: ion=Прочитайте истории успеха клие : : -: ddphrases}&keyword={keyword} : : -: operty_type]=apartment&advert_search[rooms][]=4&advert_search[price_type]=total : : -: categoryID=560&search_name=Huter&search_price_from=0&search_price_to=38625 : : -: 475 : : -: pe}&position={position}&campid=komplekt_yandex_poisk : : -: ={keyword} : : -: =28 : : -: MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=§or=&ring=&clear_cache=Y&SORT_BY=0utm_source=Yandex_CPC : : -: 06-01:2013-06-27&group=day&d1=01&cmd=showCampStat&ulogin=elama-13666725&target_all=yes&phrasedate=yes&online_stat=0&m1=06&cid=6887844&goals=0&offline_stat=0& : : -: y2=13 : : -: n} : : -: n=true : : -: n=true : : -: 2013-06-27&m2=06&group=month&d1=01&cmd=showCampStat&ulogin=beritedengi-2013&target_all=yes&phrasedate=yes&m1=06&online_stat=0&cid=6870145&goals=0&offline_sta : : -: t=0&y2=13 : : -: ice%3D0;35000%26price_from%3D0%26price_to%3D35000%26FILTER[956][]%3D1%26data_form%3DY : : -: ARE_MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=&OBJ_PERIOD_MIN=1&OBJ_PERIOD_MAX=12&SORT_BY=0&names=2574 : : -: 10000&search_subcats=1&= Найти &action=search : : -: _MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=&OBJ_DISTANCE_MIN=&OBJ_DISTANCE_MAX=&SORT_BY=0&names=3158,3669 : : -: 2013-07-01&m2=06&group=week&d1=24&cmd=showCampStat&ulogin=sumki-rekurs&target_all=yes&phrasedate=yes&m1=06&online_stat=0&cid=4054100&goals=0&offline_stat=0&y : : -: 2=13 : : -: d=dirOuyaConsole : : -: 2013-06-24&m2=07&group=year&d1=23&cmd=showCampStat&target_all=yes&phrasedate=yes&online_stat=0&m1=11&cid=4272756&goals=0&offline_stat=0&y2=13 : : -+---------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ -1000 tuples (370.979ms) -sql>\q -monetdbd is running - -times: 1 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageVi ews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -Connection terminated -monetdbd is running - -times: 2 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -unsupported hash algorithms: gorithms: gori -monetdbd is running - -times: 3 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -unsupported hash algorithms: gorithms: gorit -monetdbd is running - -times: 1 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND T raficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (1.1s) -sql>\q -monetdbd is running - -times: 2 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND T raficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (27.731ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND T raficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (22.295ms) -sql>\q -monetdbd is running - -times: 1 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AN D NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (795.122ms) -sql>\q -monetdbd is running - -times: 2 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AN D NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (23.845ms) -sql>\q -monetdbd is running - -times: 3 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AN D NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (25.664ms) -sql>\q -monetdbd is running - -times: 1 -query: SELECT EventTime - INTERVAL SECOND(EventTime) AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - INTERVAL SECOND(EventTime) AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '201 3-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -syntax error, unexpected SECOND, expecting SCOLON in: "select eventtime - interval second" -sql>\q -monetdbd is running - -times: 2 -query: SELECT EventTime - INTERVAL SECOND(EventTime) AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - INTERVAL SECOND(EventTime) AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '201 3-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -syntax error, unexpected SECOND, expecting SCOLON in: "select eventtime - interval second" -sql>\q -monetdbd is running - -times: 3 -query: SELECT EventTime - INTERVAL SECOND(EventTime) AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - INTERVAL SECOND(EventTime) AS Minute, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '201 3-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; -syntax error, unexpected SECOND, expecting SCOLON in: "select eventtime - interval second" -sql>\q -monetdbd is running -stop time: Пн. сент. 2 21:03:06 MSK 2013 diff --git a/benchmark/monetdb/log/log_10m_corrected b/benchmark/monetdb/log/log_10m_corrected deleted file mode 100644 index fe12806baf7..00000000000 --- a/benchmark/monetdb/log/log_10m_corrected +++ /dev/null @@ -1,5498 +0,0 @@ -start time: Пн. сент. 16 14:15:41 MSK 2013 -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT count(*) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m; -+----------+ -+==========+ -+----------+ -1 tuple (3.396ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT count(*) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m; -+----------+ -+==========+ -+----------+ -1 tuple (3.369ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT count(*) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m; -+----------+ -+==========+ -+----------+ -1 tuple (3.150ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID <> 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE AdvEngineID <> 0; -+--------+ -+========+ -+--------+ -1 tuple (43.776ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID <> 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE AdvEngineID <> 0; -+--------+ -+========+ -+--------+ -1 tuple (7.608ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID <> 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE AdvEngineID <> 0; -+--------+ -+========+ -+--------+ -1 tuple (7.683ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+---------+----------+--------------------------+ -+=========+==========+==========================+ -+---------+----------+--------------------------+ -1 tuple (256.328ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+---------+----------+--------------------------+ -+=========+==========+==========================+ -+---------+----------+--------------------------+ -1 tuple (24.573ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+---------+----------+--------------------------+ -+=========+==========+==========================+ -+---------+----------+--------------------------+ -1 tuple (23.258ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT sum(UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_10m; -overflow in calculation. -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT sum(UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_10m; -overflow in calculation. -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT sum(UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_10m; -overflow in calculation. -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (694.896ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (617.748ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (569.812ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (3.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (2.7s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (2.9s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_10m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (221.339ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_10m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (8.113ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_10m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (8.619ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+-------+ -+=============+=======+ -+-------------+-------+ -9 tuples (33.757ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+-------+ -+=============+=======+ -+-------------+-------+ -9 tuples (7.711ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -+-------------+-------+ -+=============+=======+ -+-------------+-------+ -9 tuples (8.596ms) -sql>\q --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+==========+========+ -+----------+--------+ -10 tuples (7.4s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+==========+========+ -+----------+--------+ -10 tuples (5.7s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -+----------+--------+ -+==========+========+ -+----------+--------+ -10 tuples (5.9s) -sql>\q --- агрегация, среднее количество ключей.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -+----------+--------+---------+--------------------------+--------+ -+==========+========+=========+==========================+========+ -+----------+--------+---------+--------------------------+--------+ -10 tuples (6.4s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -+----------+--------+---------+--------------------------+--------+ -+==========+========+=========+==========================+========+ -+----------+--------+---------+--------------------------+--------+ -10 tuples (8.6s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -+----------+--------+---------+--------------------------+--------+ -+==========+========+=========+==========================+========+ -+----------+--------+---------+--------------------------+--------+ -10 tuples (8.4s) -sql>\q --- агрегация, среднее количество ключей, несколько агрегатных функций.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+-------+ -+==================+=======+ -+------------------+-------+ -10 tuples (342.014ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+-------+ -+==================+=======+ -+------------------+-------+ -10 tuples (335.044ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -+------------------+-------+ -+==================+=======+ -+------------------+-------+ -10 tuples (583.903ms) -sql>\q --- мощная фильтрация по строкам, затем агрегация по строкам.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC L IMIT 10; -+-------------+------------------+-------+ -+=============+==================+=======+ -+-------------+------------------+-------+ -10 tuples (353.099ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC L IMIT 10; -+-------------+------------------+-------+ -+=============+==================+=======+ -+-------------+------------------+-------+ -10 tuples (337.283ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC L IMIT 10; -+-------------+------------------+-------+ -+=============+==================+=======+ -+-------------+------------------+-------+ -10 tuples (331.452ms) -sql>\q --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------+------+ -+=======================+======+ -+-----------------------+------+ -10 tuples (6.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------+------+ -+=======================+======+ -+-----------------------+------+ -10 tuples (7.0s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------+------+ -+=======================+======+ -+-----------------------+------+ -10 tuples (12.7s) -sql>\q --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+-----------------------+------+ -+=======================+======+ -+-----------------------+------+ -10 tuples (5.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+-----------------------+------+ -+=======================+======+ -+-----------------------+------+ -10 tuples (6.7s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+-----------------------+------+ -+=======================+======+ -+-----------------------+------+ -10 tuples (6.1s) -sql>\q --- агрегация чуть сложнее.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+----------------+-----------------------+------+ -+================+=======================+======+ -+----------------+-----------------------+------+ -10 tuples (59.9s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+----------------+-----------------------+------+ -+================+=======================+======+ -+----------------+-----------------------+------+ -10 tuples (56.9s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+----------------+-----------------------+------+ -+================+=======================+======+ -+----------------+-----------------------+------+ -10 tuples (1m 10s) -sql>\q --- агрегация по числу и строке, большое количество ключей.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+ -+=====================+======+ -+---------------------+------+ -10 tuples (731.583ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+ -+=====================+======+ -+---------------------+------+ -10 tuples (720.496ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+ -+=====================+======+ -+---------------------+------+ -10 tuples (707.563ms) -sql>\q --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+------+ -+=====================+==============+======+ -+---------------------+--------------+------+ -10 tuples (6.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+------+ -+=====================+==============+======+ -+---------------------+--------------+------+ -10 tuples (6.0s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+--------------+------+ -+=====================+==============+======+ -+---------------------+--------------+------+ -10 tuples (6.7s) -sql>\q --- ещё более сложная агрегация.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+---------------------+------------------------------------+------+ -+=====================+====================================+======+ -+---------------------+------------------------------------+------+ -10 tuples (6.0s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+---------------------+------------------------------------+------+ -+=====================+====================================+======+ -+---------------------+------------------------------------+------+ -10 tuples (6.3s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -+---------------------+------------------------------------+------+ -+=====================+====================================+======+ -+---------------------+------------------------------------+------+ -10 tuples (11.5s) -sql>\q --- то же самое, но без сортировки.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+------+ -+=====================+======+==============+======+ -+---------------------+------+--------------+------+ -10 tuples (23.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+------+ -+=====================+======+==============+======+ -+---------------------+------+--------------+------+ -10 tuples (37.9s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+---------------------+------+--------------+------+ -+=====================+======+==============+======+ -+---------------------+------+--------------+------+ -10 tuples (35.4s) -sql>\q --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -+--------+ -+========+ -+--------+ -0 tuples (340.345ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -+--------+ -+========+ -+--------+ -0 tuples (0.722ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -+--------+ -+========+ -+--------+ -0 tuples (0.688ms) -sql>\q --- мощная фильтрация по столбцу типа UInt64.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+------+ -+======+ -+------+ -1 tuple (2.5s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+------+ -+======+ -+------+ -1 tuple (52.693ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+------+ -+======+ -+------+ -1 tuple (47.070ms) -sql>\q --- фильтрация по поиску подстроки в строке.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+------------------------------------------------------------------------+---------------------------------------------------------------------------------------+------+ -+========================================================================+=======================================================================================+======+ -: : ya : : -+------------------------------------------------------------------------+---------------------------------------------------------------------------------------+------+ -10 tuples (94.266ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+------------------------------------------------------------------------+---------------------------------------------------------------------------------------+------+ -+========================================================================+=======================================================================================+======+ -: : ya : : -+------------------------------------------------------------------------+---------------------------------------------------------------------------------------+------+ -10 tuples (52.012ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+------------------------------------------------------------------------+---------------------------------------------------------------------------------------+------+ -+========================================================================+=======================================================================================+======+ -: : ya : : -+------------------------------------------------------------------------+---------------------------------------------------------------------------------------+------+ -10 tuples (57.737ms) -sql>\q --- вынимаем большие столбцы, фильтрация по строке.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL <> '%.yandex.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL <> '%.yandex.%' AND SearchPhrase < > '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+------------------+-----------------------------------------------------+------------------------------------------------+------+------+ -+==================+=====================================================+================================================+======+======+ -+------------------+-----------------------------------------------------+------------------------------------------------+------+------+ -10 tuples (2.8s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL <> '%.yandex.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL <> '%.yandex.%' AND SearchPhrase < > '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+------------------+-----------------------------------------------------+------------------------------------------------+------+------+ -+==================+=====================================================+================================================+======+======+ -+------------------+-----------------------------------------------------+------------------------------------------------+------+------+ -10 tuples (97.284ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL <> '%.yandex.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL <> '%.yandex.%' AND SearchPhrase < > '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+------------------+-----------------------------------------------------+------------------------------------------------+------+------+ -+==================+=====================================================+================================================+======+======+ -+------------------+-----------------------------------------------------+------------------------------------------------+------+------+ -10 tuples (93.373ms) -sql>\q --- чуть больше столбцы.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------+--------------+------+---------------+------------+------+------------+------+---------------------+------+------+------+----------------+ -: : enab : : even : : : teri : : onid : : terc : : agen : :> -: : le : : t : : : d : : : : lass : : t : :> -+=====================+======+==============+======+===============+============+======+============+======+=====================+======+======+======+================+ -: : : : : :58:04.000000 : : : : : : : : : .yandex.ru/reg : -: : : : : : : : : : : : : : istered/main.p : -: : : : : : : : : : : : : : l?cmd=newCam...> -: : : : : :58:13.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :00:43.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :01:30.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :01:57.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :02:21.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :02:29.000000 : : : : : : : : : .yandex.ru/reg : -: : : : : : : : : : : : : : istered/main.p : -: : : : : : : : : : : : : : l?from=metrika : -: : : : : :02:37.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :04:10.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :04:45.000000 : : : : : : : : : .yandex.ru/ : -+---------------------+------+--------------+------+---------------+------------+------+------------+------+---------------------+------+------+------+----------------+ -10 tuples (14.5s) !92 columns dropped, 1 field truncated! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------+--------------+------+---------------+------------+------+------------+------+---------------------+------+------+------+----------------+ -: : enab : : even : : : teri : : onid : : terc : : agen : :> -: : le : : t : : : d : : : : lass : : t : :> -+=====================+======+==============+======+===============+============+======+============+======+=====================+======+======+======+================+ -: : : : : :58:04.000000 : : : : : : : : : .yandex.ru/reg : -: : : : : : : : : : : : : : istered/main.p : -: : : : : : : : : : : : : : l?cmd=newCam...> -: : : : : :58:13.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :00:43.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :01:30.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :01:57.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :02:21.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :02:29.000000 : : : : : : : : : .yandex.ru/reg : -: : : : : : : : : : : : : : istered/main.p : -: : : : : : : : : : : : : : l?from=metrika : -: : : : : :02:37.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :04:10.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :04:45.000000 : : : : : : : : : .yandex.ru/ : -+---------------------+------+--------------+------+---------------+------------+------+------------+------+---------------------+------+------+------+----------------+ -10 tuples (502.916ms) !92 columns dropped, 1 field truncated! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -+---------------------+------+--------------+------+---------------+------------+------+------------+------+---------------------+------+------+------+----------------+ -: : enab : : even : : : teri : : onid : : terc : : agen : :> -: : le : : t : : : d : : : : lass : : t : :> -+=====================+======+==============+======+===============+============+======+============+======+=====================+======+======+======+================+ -: : : : : :58:04.000000 : : : : : : : : : .yandex.ru/reg : -: : : : : : : : : : : : : : istered/main.p : -: : : : : : : : : : : : : : l?cmd=newCam...> -: : : : : :58:13.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :00:43.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :01:30.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :01:57.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :02:21.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :02:29.000000 : : : : : : : : : .yandex.ru/reg : -: : : : : : : : : : : : : : istered/main.p : -: : : : : : : : : : : : : : l?from=metrika : -: : : : : :02:37.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :04:10.000000 : : : : : : : : : .yandex.ru/ : -: : : : : :04:45.000000 : : : : : : : : : .yandex.ru/ : -+---------------------+------+--------------+------+---------------+------------+------+------------+------+---------------------+------+------+------+----------------+ -10 tuples (527.883ms) !92 columns dropped, 1 field truncated! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q --- плохой запрос - вынимаем все столбцы.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -+------------------------------------+ -+====================================+ -+------------------------------------+ -10 tuples (242.014ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -+------------------------------------+ -+====================================+ -+------------------------------------+ -10 tuples (113.946ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -+------------------------------------+ -+====================================+ -+------------------------------------+ -10 tuples (34.256ms) -sql>\q --- большая сортировка.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -+------------------------------------------------------------------------------+ -+==============================================================================+ -+------------------------------------------------------------------------------+ -10 tuples (60.398ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -+------------------------------------------------------------------------------+ -+==============================================================================+ -+------------------------------------------------------------------------------+ -10 tuples (31.705ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -+------------------------------------------------------------------------------+ -+==============================================================================+ -+------------------------------------------------------------------------------+ -10 tuples (27.748ms) -sql>\q --- большая сортировка по строкам.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+------------------------------------+ -+====================================+ -+------------------------------------+ -10 tuples (74.668ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+------------------------------------+ -+====================================+ -+------------------------------------+ -10 tuples (44.987ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -+------------------------------------+ -+====================================+ -+------------------------------------+ -10 tuples (34.549ms) -sql>\q --- большая сортировка по кортежу.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL <> '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL <> '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -+-----------+--------------------------+---------+ -+===========+==========================+=========+ -+-----------+--------------------------+---------+ -19 tuples (2.8s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL <> '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL <> '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -+-----------+--------------------------+---------+ -+===========+==========================+=========+ -+-----------+--------------------------+---------+ -19 tuples (2.7s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL <> '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL <> '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -+-----------+--------------------------+---------+ -+===========+==========================+=========+ -+-----------+--------------------------+---------+ -19 tuples (2.7s) -sql>\q --- считаем средние длины URL для крупных счётчиков.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) -more>\q -more>monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) -more>\q -more>monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) -more>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -could not connect to localhost:50000: Connection refused -restart server: /etc/init.d/monetdb5-sql restart -Stopping MonetDB SQL server: monetdbd. -Starting MonetDB SQL server: monetdbd. - -times: 1 -query: AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -syntax error, unexpected AS in: "as" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -syntax error, unexpected AS in: "as" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -syntax error, unexpected AS in: "as" -sql>\q --- то же самое, но с разбивкой по доменам.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(Re solutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWid th + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18 ), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum( ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(Resolut ionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidt h + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43) , sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(R esolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(Resoluti onWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(Re solutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(Resolutio nWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+ -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -1 tuple (1.6s) !78 columns dropped! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(Re solutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWid th + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18 ), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum( ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(Resolut ionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidt h + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43) , sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(R esolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(Resoluti onWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(Re solutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(Resolutio nWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+ -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -1 tuple (1.2s) !78 columns dropped! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(Re solutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWid th + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18 ), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum( ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(Resolut ionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidt h + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43) , sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(R esolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(Resoluti onWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(Re solutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(Resolutio nWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+ -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -1 tuple (1.2s) !78 columns dropped! -note: to disable dropping columns and/or truncating fields use \w-1 -sql>\q --- много тупых агрегатных функций.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -Connection terminated -monetdbd is running -spawn mclient -u monetdb -d hits -password: -could not connect to localhost:50000: Connection refused -restart server: /etc/init.d/monetdb5-sql restart -Stopping MonetDB SQL server: monetdbd. -Starting MonetDB SQL server: monetdbd. - -times: 2 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -Connection terminated -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -Connection terminated --- сложная агрегация, для больших таблиц может не хватить оперативки.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Connection terminated -restart server: /etc/init.d/monetdb5-sql restart -monetdbd not running. -Starting MonetDB SQL server: monetdbd. - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (1.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (424.371ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (244.552ms) -sql>\q --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (1.3s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (1.3s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (1.3s) -sql>\q --- то же самое, но ещё и без фильтрации.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------+--------+ -+==============================================================+========+ -+--------------------------------------------------------------+--------+ -10 tuples (2m 10s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------+--------+ -+==============================================================+========+ -+--------------------------------------------------------------+--------+ -10 tuples (2m 9s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -+--------------------------------------------------------------+--------+ -+==============================================================+========+ -+--------------------------------------------------------------+--------+ -10 tuples (1m 43s) -sql>\q --- агрегация по URL.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected sqlINT in: "select 1, url, count(*) from hits_10m group by 1" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected sqlINT in: "select 1, url, count(*) from hits_10m group by 1" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected sqlINT in: "select 1, url, count(*) from hits_10m group by 1" -sql>\q --- агрегация по URL и числу.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_10m group by clientip, clientip -" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_10m group by clientip, clientip -" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_10m group by clientip, clientip -" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT R efresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------+-----------+ -+=================================================================================+===========+ -+---------------------------------------------------------------------------------+-----------+ -10 tuples (3m 30s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT R efresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------+-----------+ -+=================================================================================+===========+ -+---------------------------------------------------------------------------------+-----------+ -10 tuples (3m 24s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT R efresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -+---------------------------------------------------------------------------------+-----------+ -+=================================================================================+===========+ -+---------------------------------------------------------------------------------+-----------+ -10 tuples (3m 54s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refr esh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -\q -+------------------------------------------------+-----------+ -+================================================+===========+ -+------------------------------------------------+-----------+ -10 tuples (16m 14s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refr esh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+------------------------------------------------+-----------+ -+================================================+===========+ -+------------------------------------------------+-----------+ -10 tuples (4m 36s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refr esh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+------------------------------------------------+-----------+ -+================================================+===========+ -+------------------------------------------------+-----------+ -10 tuples (3m 26s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT I sDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+---------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ -: : iews : -+===============================================================================================================================================================+=======+ -: c_aft_spring-offer2013:cpc:yandex-direct : : -: =true : : -: hotelClass=all&meal=all&priceFrom=10000&priceTo=300000 : : -: 475 : : -: q2-2013:cpc:yandex-direct:apec2013 : : -: ks:cpc:yandex-direct:tyagach : : -: 0_pkw_stock:cpc:yandex-direct:e200 : : -: {keyword} : : -: 68ae4aada9d8779c3d9db2a299e : : -: apartment&search_form[currency]=RUB&search_form[price_type]=total&search_form[rooms][]=4&search_form[only_with_photo]=true : : -: BJ_PRICE_MIN=&OBJ_PRICE_MAX=§or=&ring=&SORT_BY=0utm_source=Yandex_CPC : : -: hotelClass=all&meal=all&priceFrom=10000&priceTo=300000 : : -: n=true : : -: -01:2013-07-01&m2=07&stat_type=custom&group_by=banner, page, phrase&onpage=100&d1=02&target_1=0&cmd=showCampStat&filter_page_target=context&reverse=1&target_ : : -: all=1&sort=aconv&m1=07&online_stat=1&cid=4828884&goals=0&y2=13&offline_stat=0 : : -: yagach : : -: tml?csref=_030_pkw_stock:cpc:yandex-direct:a-class : : -: _q2-2013:cpc:yandex-direct:edition-c : : -: =day&onpage=10000&group_by=page, phrase, retargeting, position, image, date&d1=07&target_1=0&cmd=showCampStat&reverse=ARRAY(0x14a25080)&target_all=1&m1=07&on : : -: line_stat=0&cid=7088688&goals=0&offline_stat=0&y2=13 : : -: ion=Прочитайте истории успеха клие : : -: ddphrases}&keyword={keyword} : : -: operty_type]=apartment&advert_search[rooms][]=4&advert_search[price_type]=total : : -: categoryID=560&search_name=Huter&search_price_from=0&search_price_to=38625 : : -: 475 : : -: pe}&position={position}&campid=komplekt_yandex_poisk : : -: ={keyword} : : -: =28 : : -: MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=§or=&ring=&clear_cache=Y&SORT_BY=0utm_source=Yandex_CPC : : -: 06-01:2013-06-27&group=day&d1=01&cmd=showCampStat&ulogin=elama-13666725&target_all=yes&phrasedate=yes&online_stat=0&m1=06&cid=6887844&goals=0&offline_stat=0& : : -: y2=13 : : -: n} : : -: n=true : : -: n=true : : -: 2013-06-27&m2=06&group=month&d1=01&cmd=showCampStat&ulogin=beritedengi-2013&target_all=yes&phrasedate=yes&m1=06&online_stat=0&cid=6870145&goals=0&offline_sta : : -: t=0&y2=13 : : -: ice%3D0;35000%26price_from%3D0%26price_to%3D35000%26FILTER[956][]%3D1%26data_form%3DY : : -: ARE_MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=&OBJ_PERIOD_MIN=1&OBJ_PERIOD_MAX=12&SORT_BY=0&names=2574 : : -: 10000&search_subcats=1&= Найти &action=search : : -: _MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=&OBJ_DISTANCE_MIN=&OBJ_DISTANCE_MAX=&SORT_BY=0&names=3158,3669 : : -: 2013-07-01&m2=06&group=week&d1=24&cmd=showCampStat&ulogin=sumki-rekurs&target_all=yes&phrasedate=yes&m1=06&online_stat=0&cid=4054100&goals=0&offline_stat=0&y : : -: 2=13 : : -: d=dirOuyaConsole : : -: 2013-06-24&m2=07&group=year&d1=23&cmd=showCampStat&target_all=yes&phrasedate=yes&online_stat=0&m1=11&cid=4272756&goals=0&offline_stat=0&y2=13 : : -+---------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ -1000 tuples (3.2s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT I sDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+---------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ -: : iews : -+===============================================================================================================================================================+=======+ -: c_aft_spring-offer2013:cpc:yandex-direct : : -: =true : : -: hotelClass=all&meal=all&priceFrom=10000&priceTo=300000 : : -: 475 : : -: q2-2013:cpc:yandex-direct:apec2013 : : -: ks:cpc:yandex-direct:tyagach : : -: 0_pkw_stock:cpc:yandex-direct:e200 : : -: {keyword} : : -: 68ae4aada9d8779c3d9db2a299e : : -: apartment&search_form[currency]=RUB&search_form[price_type]=total&search_form[rooms][]=4&search_form[only_with_photo]=true : : -: BJ_PRICE_MIN=&OBJ_PRICE_MAX=§or=&ring=&SORT_BY=0utm_source=Yandex_CPC : : -: hotelClass=all&meal=all&priceFrom=10000&priceTo=300000 : : -: n=true : : -: -01:2013-07-01&m2=07&stat_type=custom&group_by=banner, page, phrase&onpage=100&d1=02&target_1=0&cmd=showCampStat&filter_page_target=context&reverse=1&target_ : : -: all=1&sort=aconv&m1=07&online_stat=1&cid=4828884&goals=0&y2=13&offline_stat=0 : : -: yagach : : -: tml?csref=_030_pkw_stock:cpc:yandex-direct:a-class : : -: _q2-2013:cpc:yandex-direct:edition-c : : -: =day&onpage=10000&group_by=page, phrase, retargeting, position, image, date&d1=07&target_1=0&cmd=showCampStat&reverse=ARRAY(0x14a25080)&target_all=1&m1=07&on : : -: line_stat=0&cid=7088688&goals=0&offline_stat=0&y2=13 : : -: ion=Прочитайте истории успеха клие : : -: ddphrases}&keyword={keyword} : : -: operty_type]=apartment&advert_search[rooms][]=4&advert_search[price_type]=total : : -: categoryID=560&search_name=Huter&search_price_from=0&search_price_to=38625 : : -: 475 : : -: pe}&position={position}&campid=komplekt_yandex_poisk : : -: ={keyword} : : -: =28 : : -: MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=§or=&ring=&clear_cache=Y&SORT_BY=0utm_source=Yandex_CPC : : -: 06-01:2013-06-27&group=day&d1=01&cmd=showCampStat&ulogin=elama-13666725&target_all=yes&phrasedate=yes&online_stat=0&m1=06&cid=6887844&goals=0&offline_stat=0& : : -: y2=13 : : -: n} : : -: n=true : : -: n=true : : -: 2013-06-27&m2=06&group=month&d1=01&cmd=showCampStat&ulogin=beritedengi-2013&target_all=yes&phrasedate=yes&m1=06&online_stat=0&cid=6870145&goals=0&offline_sta : : -: t=0&y2=13 : : -: ice%3D0;35000%26price_from%3D0%26price_to%3D35000%26FILTER[956][]%3D1%26data_form%3DY : : -: ARE_MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=&OBJ_PERIOD_MIN=1&OBJ_PERIOD_MAX=12&SORT_BY=0&names=2574 : : -: 10000&search_subcats=1&= Найти &action=search : : -: _MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=&OBJ_DISTANCE_MIN=&OBJ_DISTANCE_MAX=&SORT_BY=0&names=3158,3669 : : -: 2013-07-01&m2=06&group=week&d1=24&cmd=showCampStat&ulogin=sumki-rekurs&target_all=yes&phrasedate=yes&m1=06&online_stat=0&cid=4054100&goals=0&offline_stat=0&y : : -: 2=13 : : -: d=dirOuyaConsole : : -: 2013-06-24&m2=07&group=year&d1=23&cmd=showCampStat&target_all=yes&phrasedate=yes&online_stat=0&m1=11&cid=4272756&goals=0&offline_stat=0&y2=13 : : -+---------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ -1000 tuples (519.642ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT I sDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -+---------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ -: : iews : -+===============================================================================================================================================================+=======+ -: c_aft_spring-offer2013:cpc:yandex-direct : : -: =true : : -: hotelClass=all&meal=all&priceFrom=10000&priceTo=300000 : : -: 475 : : -: q2-2013:cpc:yandex-direct:apec2013 : : -: ks:cpc:yandex-direct:tyagach : : -: 0_pkw_stock:cpc:yandex-direct:e200 : : -: {keyword} : : -: 68ae4aada9d8779c3d9db2a299e : : -: apartment&search_form[currency]=RUB&search_form[price_type]=total&search_form[rooms][]=4&search_form[only_with_photo]=true : : -: BJ_PRICE_MIN=&OBJ_PRICE_MAX=§or=&ring=&SORT_BY=0utm_source=Yandex_CPC : : -: hotelClass=all&meal=all&priceFrom=10000&priceTo=300000 : : -: n=true : : -: -01:2013-07-01&m2=07&stat_type=custom&group_by=banner, page, phrase&onpage=100&d1=02&target_1=0&cmd=showCampStat&filter_page_target=context&reverse=1&target_ : : -: all=1&sort=aconv&m1=07&online_stat=1&cid=4828884&goals=0&y2=13&offline_stat=0 : : -: yagach : : -: tml?csref=_030_pkw_stock:cpc:yandex-direct:a-class : : -: _q2-2013:cpc:yandex-direct:edition-c : : -: =day&onpage=10000&group_by=page, phrase, retargeting, position, image, date&d1=07&target_1=0&cmd=showCampStat&reverse=ARRAY(0x14a25080)&target_all=1&m1=07&on : : -: line_stat=0&cid=7088688&goals=0&offline_stat=0&y2=13 : : -: ion=Прочитайте истории успеха клие : : -: ddphrases}&keyword={keyword} : : -: operty_type]=apartment&advert_search[rooms][]=4&advert_search[price_type]=total : : -: categoryID=560&search_name=Huter&search_price_from=0&search_price_to=38625 : : -: 475 : : -: pe}&position={position}&campid=komplekt_yandex_poisk : : -: ={keyword} : : -: =28 : : -: MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=§or=&ring=&clear_cache=Y&SORT_BY=0utm_source=Yandex_CPC : : -: 06-01:2013-06-27&group=day&d1=01&cmd=showCampStat&ulogin=elama-13666725&target_all=yes&phrasedate=yes&online_stat=0&m1=06&cid=6887844&goals=0&offline_stat=0& : : -: y2=13 : : -: n} : : -: n=true : : -: n=true : : -: 2013-06-27&m2=06&group=month&d1=01&cmd=showCampStat&ulogin=beritedengi-2013&target_all=yes&phrasedate=yes&m1=06&online_stat=0&cid=6870145&goals=0&offline_sta : : -: t=0&y2=13 : : -: ice%3D0;35000%26price_from%3D0%26price_to%3D35000%26FILTER[956][]%3D1%26data_form%3DY : : -: ARE_MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=&OBJ_PERIOD_MIN=1&OBJ_PERIOD_MAX=12&SORT_BY=0&names=2574 : : -: 10000&search_subcats=1&= Найти &action=search : : -: _MAX=&OBJ_PRICE_MIN=&OBJ_PRICE_MAX=&OBJ_DISTANCE_MIN=&OBJ_DISTANCE_MAX=&SORT_BY=0&names=3158,3669 : : -: 2013-07-01&m2=06&group=week&d1=24&cmd=showCampStat&ulogin=sumki-rekurs&target_all=yes&phrasedate=yes&m1=06&online_stat=0&cid=4054100&goals=0&offline_stat=0&y : : -: 2=13 : : -: d=dirOuyaConsole : : -: 2013-06-24&m2=07&group=year&d1=23&cmd=showCampStat&target_all=yes&phrasedate=yes&online_stat=0&m1=11&cid=4272756&goals=0&offline_stat=0&y2=13 : : -+---------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ -1000 tuples (654.508ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageVi ews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -: icso : chen : ngin : : : iews : -: urce : gine : eid : : : : -: id : id : : : : : -+======+======+======+====================================================================+=====================================================================+=======+ -: : : : : ype=campdate : : -: : : : type=campdate : : : -: : : : : &mediaType=text : : -: : : : : &payment_type=1 : : -: : : : : d=7 : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0 : : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : ype=campdate : : -: : : : : dirlevel=1372668697.0&cmd=showCamps : : -: : : : : 98.0&authredirlevel=1371874016.0&authredirlevel=1371612009.0&authre : : -: : : : : dirlevel=1370955451.0&cmd=showCamp&cid=5477450 : : -: : : : t : : : -: : : : 1&mediaType=text : : : -: : : : : 9&ulogin=uniparxdir : : -: : : : : owCamps : : -: : : : : 0 : : -: : : : : 41.0&authredirlevel=1373081665.0&authredirlevel=1372822331.0&authre : : -: : : : : dirlevel=1372563095.0&authredirlevel=1372303801.0&authredirlevel=13 : : -: : : : : 72044532.0&authredirlevel=1371730939.0&authredirlevel=1371471570.0& : : -: : : : : authredirlevel=1371212358.0&cmd=showCamp&cid=2856748 : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : -: : : : type=campdate : : : -: : : : : 95.0&authredirlevel=1372303801.0&authredirlevel=1372044532.0&authre : : -: : : : : dirlevel=1371730939.0&authredirlevel=1371471570.0&authredirlevel=13 : : -: : : : : 71212358.0&cmd=showCamp&cid=2856748 : : -: : : : : =225&text_geo=Россия : : -: : : : : 3 : : -: : : : 93 : : : -: : : : : 9 : : -: : : : 19 : : : -: : : : : s=1 : : -: : : : : =1&text_geo=Москва и область : : -: : : : : 8&ulogin=uniparxdir : : -: : : : : 2&ulogin=uniparxdir : : -: : : : : 5&ulogin=uniparxdir : : -: : : : : 6&ulogin=woodenhousedir : : -: : : : : 7&ulogin=uniparxdir : : -: : : : : 4&ulogin=uniparxdir : : -: : : : : 7 : : -: : : : 37 : : : -: : : : 71 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : : 5 : : -: : : : : 5 : : -: : : : 01 : : : -: : : : : 8 : : -: : : : 19 : : : -: : : : : 1 : : -: : : : 65 : : : -: : : : 29 : : : -: : : : 45 : : : -: : : : 03 : : : -: : : : : 0 : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 81 : : : -: : : : 95 : : : -: : : : 40 : : : -: : : : : 5 : : -: : : : 08 : : : -: : : : : 3 : : -: : : : : 1 : : -: : : : : 60.0&authredirlevel=1373047694.0&authredirlevel=1371465714.0&authre : : -: : : : : dirlevel=1370425373.0&authredirlevel=1366055041.0&authredirlevel=13 : : -: : : : : 66054818.0&authredirlevel=1365875147.0&authredirlevel=1365611062.0& : : -: : : : : authredirlevel=1365320979.0&authredirlevel=1365066414.0&cmd=showCam : : -: : : : : p&cid=5516805&ulogin=uniparxdir : : -: : : : : d=7 : : -: : : : : &mediaType=text : : -: : : : edirlevel=1365585299.0, 1365577945.0, 1365574277.0, 1365566946.0, : dirlevel=1365585299.0, 1365577945.0, 1365574277.0, 1365566946.0, 13 : : -: : : : 1365563277.0, 1365559610.0, 1365555942.0, 1365552277.0, 1365544943 : 65563277.0, 1365559610.0, 1365555942.0, 1365552277.0, 1365544943.0, : : -: : : : .0, 1365541249.0, 1365537561.0, 1365533881.0, 1365530189.0, 136552 : 1365541249.0, 1365537561.0, 1365533881.0, 1365530189.0, 1365526529 : : -: : : : 6529.0, 1365522844.0, 1365519158.0, 1365515476.0, 1365511812.0, 13 : .0, 1365522844.0, 1365519158.0, 1365515476.0, 1365511812.0, 1365508 : : -: : : : 65508150.0, 1365504453.0, 1365500842.0, 1365497170.0, 1365493502.0 : 150.0, 1365504453.0, 1365500842.0, 1365497170.0, 1365493502.0, 1365 : : -: : : : , 1365486155.0, 1365482482.0, 1365478812.0, 1365475137.0, 13654714 : 486155.0, 1365482482.0, 1365478812.0, 1365475137.0, 1365471458.0, 1 : : -: : : : 58.0, 1365467780.0, 1365464101.0, 1365460425.0, 1365449525.0, 1365 : 365467780.0, 1365464101.0, 1365460425.0, 1365449525.0, 1365445860.0 : : -: : : : 445860.0, 1365438567.0, 1365434899.0, 1365431229.0, 1365427527.0, : , 1365438567.0, 1365434899.0, 1365431229.0, 1365427527.0, 136542385 : : -: : : : 1365423856.0, 1365420186.0, 1365416498.0, 1365412811.0, 1365409143 : 6.0, 1365420186.0, 1365416498.0, 1365412811.0, 1365409143.0, 136540 : : -: : : : .0, 1365405479.0, 1365401787.0, 1365398118.0, 1365394453.0, 136539 : 5479.0, 1365401787.0, 1365398118.0, 1365394453.0, 1365390789.0, 136 : : -: : : : 0789.0, 1365387124.0, 1365383460.0, 1365379796.0, 1365376129.0, 13 : 5387124.0, 1365383460.0, 1365379796.0, 1365376129.0, 1365372459.0, : : -: : : : 65372459.0, 1365368759.0, 1365365050.0, 1365361381.0, 1365357704.0 : 1365368759.0, 1365365050.0, 1365361381.0, 1365357704.0, 1365350422. : : -: : : : , 1365350422.0, 1365346752.0, 1365343085.0, 1365339417.0, 13653357 : 0, 1365346752.0, 1365343085.0, 1365339417.0, 1365335750.0, 13653284 : : -: : : : 50.0, 1365328485.0, 1365324791.0, 1365320979.0, 1365264347.0, 1365 : 85.0, 1365324791.0, 1365320979.0, 1365264347.0, 1365260692.0&cid=55 : : -: : : : 260692.0&cid=5533477&cmd=showCamp&ulogin=uniparxdir : 33477&cmd=showCamp&ulogin=uniparxdir : : -: : : : : owCamps : : -: : : : 71 : : : -: : : : : 6 : : -: : : : : 3 : : -: : : : : 6 : : -: : : : : owCamps&ulogin=holodilnikru : : -: : : : : dirlevel=1342185655.0&cmd=showCamps&ulogin=beautyland-artha8 : : -: : : : : owCamps : : -: : : : : 3 : : -: : : : : 5 : : -: : : : 43 : : : -: : : : : 5&ulogin=uniparxdir : : -: : : : : 07.0&authredirlevel=1366833461.0&authredirlevel=1366055042.0&authre : : -: : : : : dirlevel=1366054816.0&cmd=showCamp&cid=6419449&ulogin=uniparxdir : : -: : : : : 6 : : -: : : : : 1 : : -: : : : : n= : : -: : : : ks=1 : : : -: : : : howCampStat&cid=5640011&target_type=&detail=yes&y1=13&m1=06&d1=17& : owCampStat&cid=5640011&target_type=&detail=yes&y1=13&m1=06&d1=17&y2 : : -: : : : y2=13&m2=07&d2=28&group=week : =13&m2=07&d2=28&group=day : : -: : : : : 70.0&cmd=showCamp&cid=2753896 : : -: : : : : 8 : : -: : : : : owCamp&cid=7065376#187618038 : : -: : : : : 60.0&authredirlevel=1373047694.0&authredirlevel=1372504061.0&authre : : -: : : : : dirlevel=1371984554.0&cmd=showCamp&cid=5532920&ulogin=uniparxdir : : -: : : : : 6 : : -: : : : 76 : : : -: : : : 78 : : : -: : : : : 8 : : -: : : : : 5 : : -: : : : 55 : : : -: : : : 21 : : : -: : : : : 1 : : -: : : : : =Yes&types=days&cid=6999588 : : -: : : : : owCamps&ulogin=holodilnikru : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : : 0 : : -: : : : howCamps : 16.0&cmd=showCamp&cid=2196876 : : -: : : : : 5 : : -: : : : type=campdate : : : -: : : : : 8 : : -: : : : : 85.0&authredirlevel=1372052820.0&authredirlevel=1371310346.0&authre : : -: : : : : dirlevel=1370938849.0&authredirlevel=1370679627.0&cmd=showCampStat& : : -: : : : : stat_type=campdate : : -: : : : : 69159&phrasedate=Yes&target_all=1 : : -: : : : : 5 : : -: : : : 75 : : : -: : : : : =easy&from=metrika : : -: : : : type=campdate : : : -: : : : : Informers.html : : -: : : : : w.advertising.yandex.ru/welcome/pdf/direct_booklet.pdf : : -: : : : : 7 : : -: : : : ks=1 : : : -: : : : 66&ulogin=merc-traffic : nz_russia/ru/home/van/home/new_vans/special_offers.html?csref=_030_ : : -: : : : : vans:cpc:yandex-direct : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=1475303 : : -: : : : 77 : lshoj-kvartiry : : -: : : : : =Yes&types=days&cid=5995157 : : -: : : : : =Yes&types=days&cid=6883763 : : -: : : : : 7 : : -: : : : d : : : -: : : : : ger : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : : 161272&cmd=showCamp : : -: : : : all : : : -: : : : : 0 : : -: : : : howCamps : 6 : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : 0&ulogin=uniparxdir : : -: : : : : 4 : : -: : : : : 9 : : -: : : : : 05.6&authredirlevel=1372014905.5&authredirlevel=1372014905.4&authre : : -: : : : : dirlevel=1372014905.3&authredirlevel=1372014905.2&authredirlevel=13 : : -: : : : : 72014905.1&authredirlevel=1372014905.0&authredirlevel=1370480737.6& : : -: : : : : authredirlevel=1370480737.5&authredirlevel=1370480737.4&authredirle : : -: : : : : vel=1370480737.3&authredirlevel=1370480737.2&authredirlevel=1370480 : : -: : : : : 737.1&authredirlevel=1370480737.0&cmd=showCampStat&stat_type=campda : : -: : : : : te&target_0=1&target_1=1&target_all=1 : : -: : : : : 6 : : -: : : : l=Yes&types=days&cid=7052441 : owCampStat&cid=7052441&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-26:2013-06-26,2013-06-27:2013-06-27,2013-06-25:2013-06-25&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&tar : : -: : : : : get_all=yes : : -: : : : l=Yes&types=days&cid=7066272 : 66272&phrasedate=Yes&target_all=1 : : -: : : : : 2 : : -: : : : : owCamps&ulogin=visacity08 : : -: : : : : 274687&cmd=showCamp : : -: : : : type=campdate : : : -: : : : l=Yes&types=days&cid=6723854 : owCampStat&cid=6723854&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-26:2013-06-26,2013-06-27:2013-06-27,2013-06-25:2013-06-25&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&tar : : -: : : : : get_all=yes : : -: : : : 42 : : : -: : : : 1&mediaType=text : : : -: : : : : owSubClientCamps : : -: : : : : =Yes&types=days&cid=7066272 : : -: : : : : camps=0&cmd=showCamps&settings_sorted=&sort=name&reverse=0 : : -: : : : : 3 : : -: : : : 56 : owCampStat&cid=5895656&detail=Yes&types=days : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6963923 : : -: : : : : 5 : : -: : : : 09 : : : -: : : : : type=campdate : : -: : : : : ype=campdate : : -: : : : : &mediaType=text&firstbanner=std : : -: : : : howCamp&cid=1971054 : dirlevel=1372767978.0&cmd=showCampStat&cid=1971054&detail=Yes&types : : -: : : : : =days : : -: : : : : owCamps : : -: : : : : 4 : : -: : : : : 5 : : -: : : : : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7134795&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : owSubClientCamps : : -: : : : : =Yes&types=days&cid=6830575 : : -: : : : 916.0&cmd=showCamp&cid=2196876 : 16.0&cid=2196876&cmd=showCamp : : -: : : : &ulogin= : ulogin= : : -: : : : : 8 : : -: : : : : come=yes : : -: : : : : ed : : -: : : : : 6 : : -: : : : : 5 : : -: : : : : 7 : : -: : : : howCamps : 5 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : : 0 : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : : 2 : : -: : : : howCampStat&cid=4588753&offline_stat=0&online_stat=1&stat_periods= : owCamps&mediaType=text&ulogin=FOREX-IMIKS : : -: : : : 2013-06-01:2013-06-30,2013-06-25:2013-07-01,2013-06-17:2013-07-02& : : : -: : : : ulogin=FOREX-IMIKS&stat_type=geo&page=&group=day&y1=2013&m1=06&d1= : : : -: : : : 01&y2=2013&m2=06&d2=30&goals=0&target_all=yes : : : -: : : : : ulogin= : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=3656357 : : -: : : : 10 : : : -: : : : 85 : : : -: : : : : dirlevel=1372758445.0&cmd=showCamp&cid=970693 : : -: : : : : owCamps : : -: : : : site_promotion_search : : : -: : : : t : : : -: : : : 19 : : : -: : : : 58 : : : -: : : : : 43738&stat_type=pages&target_all=1&d1=23&m1=07&y1=12&d2=23&m2=07&y2 : : -: : : : : =13&group=day : : -: : : : : 8 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : howClients : vancedForecast : : -: : : : 41 : : : -: : : : : 32.0&cmd=showCamp&cid=7171725 : : -: : : : 39 : : : -: : : : : 2 : : -: : : : : 6 : : -: : : : : &mediaType=text : : -: : : : : 9 : : -: : : : : 1 : : -: : : : : 6 : : -: : : : : 65759&stat_type=pages&target_all=1&d1=23&m1=07&y1=12&d2=23&m2=07&y2 : : -: : : : : =13&group=day : : -: : : : : 1 : : -: : : : 13 : owCamp&cid=7064513 : : -: : : : : 5 : : -: : : : 08 : : : -: : : : 24 : : : -: : : : : 5 : : -: : : : = : ulogin= : : -: : : : : 6 : : -: : : : : 1 : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=7052441 : owCampStat&cid=7052441&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-26:2013-06-26,2013-06-27:2013-06-27,2013-06-25:2013-06-25&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&tar : : -: : : : : get_all=yes : : -: : : : ransfer : : : -: : : : : 1 : : -: : : : 13 : : : -: : : : : 9 : : -: : : : : 9 : : -: : : : 19 : : : -: : : : : 8 : : -: : : : ncy : : : -: : : : 3243479&tab=media : owCamp&cid=3243479&tab=active : : -: : : : 27 : : : -: : : : howCampStat&cid=3375426&detail=Yes&target_all=1&types=days&show_ba : owCampStat&cid=3375426&detail=Yes&target_all=1&types=days&show_bann : : -: : : : nners_stat=1&page=2 : ers_stat=1&page=1 : : -: : : : : owCamp&cid=7006443&tab=all : : -: : : : : &mediaType=text : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 2 : : -: : : : : 9 : : -: : : : 35 : : : -: : : : 41133;detail=Yes;types=days : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=141133&sho : : -: : : : : w_banners_stat=1 : : -: : : : howCamps : 7 : : -: : : : : 1 : : -: : : : : 0 : : -: : : : howCamps : 5 : : -: : : : 57 : : : -: : : : 77&ulogin=merc-traffic : me_mpc/passengercars/home/servicesandaccessories/special_offers/spr : : -: : : : : ing_2013.html?csref=_mpc_aft_spring-offer2013:cpc:yandex-direct : : -: : : : &ulogin= : : : -: : : : 77 : : : -: : : : howCampStat&cid=2870560&offline_stat=0&online_stat=0&stat_periods= : 0 : : -: : : : 2013-07-01:2013-07-01,2013-06-23:2013-06-24,2013-06-16:2013-06-16& : : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day : : : -: : : : &goals=0&target_all=yes&show_banners_stat=1 : : : -: : : : howCampStat&cid=7022493&detail=Yes&target_all=1&types=days&show_ba : : : -: : : : nners_stat=1&page=1 : : : -: : : : type=campdate : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-18:201 : : -: : : : : 3-07-19,&stat_type=campdate&y1=2013&m1=07&d1=23&y2=2013&m2=07&d2=23 : : -: : : : : &group=day&target_all=1 : : -: : : : : 0 : : -: : : : 20 : : : -: : : : l=Yes&types=days&cid=5826713 : : : -: : : : 999588&phrasedate=Yes&target_all=1 : =Yes&cid=6999588&showCampStat=cmd&types=days&target_all=1 : : -: : : : : ulogin= : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-06-27:2 : : : -: : : : 013-06-28,2013-06-26:2013-06-27,2013-06-01:2013-06-20&stat_type=ca : : : -: : : : mpdate&y1=2013&m1=07&d1=01&y2=2013&m2=07&d2=01&group=day&target_al : : : -: : : : l=1 : : : -: : : : : 6 : : -: : : : : 8 : : -: : : : : 0 : : -: : : : l=Yes&types=days&cid=6932408 : 32408&phrasedate=Yes&target_all=1 : : -: : : : : owCamps&ulogin=holodilnikru : : -: : : : : 98.0&cmd=showCamp&cid=7067235 : : -: : : : : &mediaType=text : : -: : : : : 8 : : -: : : : : 0 : : -: : : : : 1 : : -: : : : : 0 : : -: : : : howCamp&cid=7026131 : : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : 1 : : -: : : : howCampStat&cid=4036889&detail=Yes&types=days : 36889&phrasedate=Yes&target_all=1 : : -: : : : 11 : 13411=1#list : : -: : : : 12 : : : -: : : : howCamp&cid=4246248&tab=decline : owCamp&cid=4246248&tab=active : : -: : : : : 9 : : -: : : : : 6 : : -: : : : 07 : : : -: : : : l=Yes&types=days&cid=5033617 : : : -: : : : : 0 : : -: : : : : 1 : : -: : : : : 4 : : -: : : : 1&mediaType=text : : : -: : : : : 2 : : -: : : : : 7 : : -: : : : l=Yes&types=days&cid=7034128 : : : -: : : : 99 : : : -: : : : id=7 : : : -: : : : : 8 : : -: : : : : 7 : : -: : : : : owCamps&ulogin=visamaster08 : : -: : : : 43 : : : -: : : : howCamps&ulogin=dc12a : owCampStat&cid=5658570&cid=5753133&cid=5701003&stat_type=campdate&u : : -: : : : : login=dc12a&csrf_token=shocsLKxtAnjy3K1 : : -: : : : : 0 : : -: : : : : owCamps&ulogin=TENDER-SELDON-IMIKS : : -: : : : : 73.0&cmd=showCampStat&cid=6421801&cid=6424294&cid=6487283&cid=64938 : : -: : : : : 29&cid=6494462&cid=6495189&stat_type=campdate&csrf_token=pRFezf5GIV : : -: : : : : r0wz8J : : -: : : : 77 : : : -: : : : : =Yes&types=days&cid=4537562 : : -: : : : : =Yes&types=days&cid=3353427 : : -: : : : 01 : : : -: : : : : 7 : : -: : : : : 2 : : -: : : : 52 : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6932408 : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=7162500 : : : -: : : : howCampStat&cid=1714344&offline_stat=0&online_stat=0&stat_periods= : owCampStat&cid=1714344&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : 2013-06-01:2013-06-30,2013-06-28:2013-06-29,2013-06-27:2013-06-28& : 13-06-01:2013-06-30,2013-06-28:2013-06-29,2013-06-27:2013-06-28&det : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&goa : : -: : : : &goals=0&target_all=yes&show_banners_stat=1 : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : : vancedForecast : : -: : : : : 3 : : -: : : : : 4 : : -: : : : 66 : : : -: : : : : 1 : : -: : : : howCampStat&cid=5360801&offline_stat=0&online_stat=1&stat_periods= : : : -: : : : &types=total&page=&group=day&y1=2012&m1=07&d1=02&y2=2013&m2=07&d2= : : : -: : : : 02 : : : -: : : : : 5 : : -: : : : t : mpleCamp : : -: : : : : 5 : : -: : : : : 2 : : -: : : : : 5 : : -: : : : : 6 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 4 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=7032884 : : -: : : : ts : : : -: : : : howCamp&cid=6411626&tab=off : owCamp&cid=6411626 : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : : 3 : : -: : : : : 6 : : -: : : : : 5 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6844274 : : -: : : : l=Yes&cid=2139196&ulogin=onclinic-sm&showCampStat=cmd&types=days&t : owCampStat&cid=2139196&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : arget_all=1&d1=01&m1=06&y1=13&d2=01&m2=07&y2=13&group=day : 13-06-01:2013-07-02,2013-06-21:2013-07-02,2013-06-24:2013-07-02&ulo : : -: : : : : gin=onclinic-sm&detail=yes&page=&y1=2013&m1=06&d1=01&y2=2013&m2=07& : : -: : : : : d2=02&group=day&goals=0&target_all=yes : : -: : : : : 6 : : -: : : : : 1 : : -: : : : 75 : : : -: : : : : 6 : : -: : : : 148542&stat_type=pages&target_all=1 : =Yes&cid=7148542&showCampStat=cmd&types=days&target_all=1 : : -: : : : : owCampStat&cid=4917291&detail=Yes&types=days : : -: : : : : 1 : : -: : : : 7-.31620-1.1372703844295129.198343986.198343986..53121502.1406.512 : 7 : : -: : : : l=Yes&types=days&cid=7041629 : : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6027971 : : -: : : : 20 : : : -: : : : : vancedForecast : : -: : : : : =Yes&types=days&cid=7034128 : : -: : : : =total&target_all=1&page=1&cmd=showCampStat&cid=3016175&show_banne : owCampStat&cid=3016175&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : rs_stat=1 : 12-02-03:2012-02-04,2011-08-01:2011-09-04,2010-12-29:2011-01-02&typ : : -: : : : : es=total&page=&group=day&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 13411=1#list : : -: : : : l=Yes&types=days&cid=88689 : : : -: : : : : 88.0 : : -: : : : howCamps&mediaType=text&ulogin=super-svictor : ype=campdate&&ulogin=super-svictor : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=6940624 : : -: : : : : &mediaType=text&firstbanner=easy : : -: : : : : dirlevel=1372663385.0&cmd=showCamp&cid=2139196&ulogin=onclinic-sm : : -: : : : 97&ulogin=mirkli-webmart : : : -: : : : : 6 : : -: : : : : 0 : : -: : : : l=Yes&types=days&cid=4644721 : owCampStat&cid=4644721&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-05-27:2013-05-28,2011-09-12:2011-09-12,2011-06-05:2011-07-05&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 5 : : -: : : : : dirlevel=1372412588.0&authredirlevel=1372408461.0&authredirlevel=13 : : -: : : : : 72403851.0&authredirlevel=1372339893.0&authredirlevel=1372335674.0& : : -: : : : : authredirlevel=1372331762.0&authredirlevel=1372323148.0&authredirle : : -: : : : : vel=1372313398.0&authredirlevel=1372309630.0&authredirlevel=1372252 : : -: : : : : 641.0&authredirlevel=1372248697.0&authredirlevel=1372244608.0&authr : : -: : : : : edirlevel=1372240548.0&authredirlevel=1372231517.0&authredirlevel=1 : : -: : : : : 372227248.0&authredirlevel=1372074352.0&authredirlevel=1371476832.0 : : -: : : : : &cmd=showCamp&tab=active&cid=6051926 : : -: : : : howCamp&cid=6501536 : : : -: : : : howCampStat&cid=7022493&detail=Yes&target_all=1&types=days&show_ba : : : -: : : : nners_stat=1&page=2 : : : -: : : : : &mediaType=text : : -: : : : : =Yes&types=days&cid=7041629 : : -: : : : : 9 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=2704830 : : : -: : : : : =Yes&types=days&cid=6100946 : : -: : : : : 0 : : -: : : : : 8 : : -: : : : : 37.0&authredirlevel=1371612009.0&authredirlevel=1370955451.0&cmd=sh : : -: : : : : owCamp&cid=5477450 : : -: : : : 18 : : : -: : : : 17 : : : -: : : : : 0 : : -: : : : : 6 : : -: : : : : dirlevel=1374312836.0&cmd=showCamps : : -: : : : : 5 : : -: : : : 57 : : : -: : : : : 5 : : -: : : : : 3 : : -: : : : l=Yes&cid=2870560&showCampStat=cmd&types=days&target_all=1&d1=02&m : 0 : : -: : : : 1=07&y1=13&d2=02&m2=07&y2=13&group=day : : : -: : : : : 0 : : -: : : : : 3 : : -: : : : &ulogin= : ulogin= : : -: : : : 12 : : : -: : : : oInformers.html : : : -: : : : howCamp&cid=4693423&tab=active : owCamp&cid=4693423&tab=wait : : -: : : : : 2 : : -: : : : : 1 : : -: : : : : 9 : : -: : : : : 11.0&authredirlevel=1372755166.0&cmd=showClients : : -: : : : : 1 : : -: : : : : 1 : : -: : : : : 3 : : -: : : : : 066835&cmd=showCamp : : -: : : : : 0 : : -: : : : 6 : : : -: : : : : 3 : : -: : : : : 7 : : -: : : : howCamp&cid=970693 : owCamp&cid=2997085 : : -: : : : : 8 : : -: : : : 1&mediaType=text : : : -: : : : : 6 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : 79 : owCamp&cid=7108679&tab=&page=2 : : -: : : : : =Yes&types=days&cid=7169384 : : -: : : : oInformers.html : : : -: : : : : dirlevel=1372772163.0&cmd=showCamps&tab=all&ulogin=spb-tester1 : : -: : : : : 8 : : -: : : : : 0 : : -: : : : 61 : : : -: : : : : vancedForecast : : -: : : : : =Yes&types=days&cid=6974488 : : -: : : : 84 : : : -: : : : : owCamp&cid=6833286 : : -: : : : : 5 : : -: : : : : 4 : : -: : : : howCamps&ulogin=major-auto-RW : ype=campdate&&ulogin=major-auto-RW : : -: : : : : 9 : : -: : : : ditBannerEasy&cid=6939766&bid=178815009 : dirlevel=1372759245.0&&cid=6939766&cmd=showCamp : : -: : : : ned : : : -: : : : ditCamp&cid=2870560 : owCamp&cid=2870560 : : -: : : : : 8 : : -: : : : howCampStat&cid=7058631&detail=Yes&types=days : 58631&stat_type=pages&target_all=1 : : -: : : : : =Yes&types=days&cid=3714884 : : -: : : : 97 : : : -: : : : : 0 : : -: : : : bids=5756034&cid=1840733 : : : -: : : : : =Yes&types=days&cid=4641928 : : -: : : : 91 : : : -: : : : : 9 : : -: : : : : 3 : : -: : : : : 49.0&authredirlevel=1372754333.0&authredirlevel=1372750347.0&cmd=sh : : -: : : : : owCamp&cid=5694785&ulogin=otzyvavto : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0#6136521 : : -: : : : l=Yes&types=days&cid=7153732 : 53732&phrasedate=Yes&target_all=1 : : -: : : : : 4 : : -: : : : : 1 : : -: : : : : 1 : : -: : : : : =213&text_geo=Москва : : -: : : : : 0 : : -: : : : : 6 : : -: : : : : ype=campdate : : -: : : : : 3 : : -: : : : : &mediaType=text&firstbanner=std : : -: : : : : 4 : : -: : : : : 8 : : -: : : : 92 : owCampStat&cid=6486492&detail=Yes&types=days : : -: : : : l=Yes&types=days&cid=6975816 : 6 : : -: : : : : owCamps : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 1 : : -: : : : 75 : : : -: : : : : =Yes&types=days&cid=6942376 : : -: : : : 68 : : : -: : : : : 7 : : -: : : : : 4 : : -: : : : : 9 : : -: : : : l=Yes&types=days&cid=6851682 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=6851682&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : =Yes&types=days&cid=2704830 : : -: : : : 21 : : : -: : : : 04 : owCampStat&cid=7057804&detail=Yes&types=days : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=6486629 : : -: : : : 73 : : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 31 : owCamp&cid=6839731 : : -: : : : 066272&phrasedate=Yes&target_all=1 : : : -: : : : howCampStat&cid=2803428&offline_stat=0&online_stat=0&stat_periods= : 8 : : -: : : : 2013-06-01:2013-06-24,2013-06-01:2013-06-14,2013-06-03:2013-06-03& : : : -: : : : phrasedate=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group : : : -: : : : =day&goals=0&target_all=yes : : : -: : : : : 5 : : -: : : : : 86.0&authredirlevel=1372751903.0&cmd=showCamp&cid=3055906 : : -: : : : : 3 : : -: : : : : 0 : : -: : : : : =Yes&types=days&cid=6157316 : : -: : : : : =Yes&types=days&cid=3502476 : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0#5809640 : : -: : : : : =Yes&types=days&cid=6860824 : : -: : : : edirlevel=1372746442.0&authredirlevel=1372714097.0&authredirlevel= : owCampStat&cid=3226358&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : 1372698041.0&authredirlevel=1372693736.0&cmd=showCampStat&cid=3226 : 13-06-24:2013-07-02,&phrasedate=yes&page=&y1=2013&m1=06&d1=24&y2=20 : : -: : : : 358&offline_stat=0&online_stat=0&stat_periods=&phrasedate=yes&page : 13&m2=07&d2=02&group=day&goals=0&target_all=yes : : -: : : : =&y1=2013&m1=06&d1=24&y2=2013&m2=07&d2=01&group=day&goals=0&target : : : -: : : : _all=yes : : : -: : : : : 5 : : -: : : : howCamps&ulogin=dc12a : owCampStat&cid=7041500&stat_type=campdate&ulogin=dc12a&csrf_token=Q : : -: : : : : un6g1847J4TOMJW : : -: : : : howCamp&tab=all&cid=6831216 : owCamp&cid=6831216&tab=active : : -: : : : : 3 : : -: : : : : 064001&cmd=showCamp : : -: : : : : =Yes&types=days&cid=7165103 : : -: : : : : 6 : : -: : : : : ype=campdate : : -: : : : 04 : : : -: : : : 622777;offline_stat=0;online_stat=1;stat_periods=2013-06-01:2013-0 : : : -: : : : 7-02,2013-04-01:2013-06-28,2013-03-25:2013-06-28;detail=yes;page=; : : : -: : : : y1=2013;m1=06;d1=01;y2=2013;m2=07;d2=02;group=day;goals=0;target_a : : : -: : : : ll=yes : : : -: : : : : =Yes&types=days&cid=7066904 : : -: : : : : 90.0&cmd=showCamp&cid=7142178 : : -: : : : : =Yes&types=days&cid=4917905 : : -: : : : 171931&phrasedate=Yes&target_all=1 : : : -: : : : : 9 : : -: : : : l=Yes&types=days&cid=5967378 : owCampStat&cid=5967378&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-18:2013-07-02,2013-06-18:2013-06-24,2013-06-24:2013-06-30&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&goa : : -: : : : : ls=0&target_all=yes : : -: : : : : =Yes&types=days&cid=5685110 : : -: : : : : 3 : : -: : : : : owCamp&cid=7163322#197090829 : : -: : : : : 1 : : -: : : : l=Yes&types=days&cid=7087520 : : : -: : : : : =Yes&types=days&cid=7064245 : : -: : : : 36 : : : -: : : : : 4 : : -: : : : : owCamp&cid=2217231 : : -: : : : : 064001&cmd=showCamp : : -: : : : : 0 : : -: : : : : 2 : : -: : : : : 4 : : -: : : : : 0 : : -: : : : : 9 : : -: : : : 68 : : : -: : : : howCamp&cid=2838772&tab=off : owCamp&cid=2838772&tab=active : : -: : : : : 8 : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=7036399 : : : -: : : : 73 : : : -: : : : howCamp&cid=6155736#165459179 : : : -: : : : : 5 : : -: : : : : 2 : : -: : : : &ulogin= : : : -: : : : : 1 : : -: : : : : 0 : : -: : : : : 8 : : -: : : : 61 : : : -: : : : l=Yes&types=days&cid=7051349 : : : -: : : : : =Yes&types=days&cid=7065579 : : -: : : : : 5 : : -: : : : 02 : : : -: : : : : 27.0&cmd=showCamp&cid=4670220 : : -: : : : : 9 : : -: : : : : 1 : : -: : : : : =Yes&types=days&cid=4764350 : : -: : : : : 6 : : -: : : : : 3 : : -: : : : 35 : owCampStat&cid=6921735&detail=Yes&types=days : : -: : : : : =Yes&types=days&cid=6851682 : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : : =Yes&types=days&cid=7055483 : : -: : : : 14 : : : -: : : : : 4 : : -: : : : 69 : : : -: : : : : 4 : : -: : : : : campdate : : -: : : : : 2 : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : ditCamp&cid=7100989 : 00989;offline_stat=0;online_stat=1;stat_periods=2013-01-10:2013-01- : : -: : : : : 28,2013-01-10:2013-01-27,2013-01-10:2013-01-26;detail=yes;page=;y1= : : -: : : : : 2013;m1=06;d1=22;y2=2013;m2=07;d2=22;group=day;goals=0;target_all=y : : -: : : : : es : : -: : : : : 2 : : -: : : : : 4 : : -: : : : 35 : : : -: : : : : 7 : : -: : : : : g14AZEA.pl?authredirlevel=1372773289.0&cmd=showCamps&ulogin=eslucha : : -: : : : : k : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : howCamps : 7 : : -: : : : 92 : : : -: : : : : 02.0&cmd=showClients : : -: : : : : 1 : : -: : : : howCamp&cid=7090606 : owCampStat&cid=7090606&detail=Yes&types=days : : -: : : : 61 : : : -: : : : : =Yes&types=days&cid=6994484 : : -: : : : : 3 : : -: : : : : 4 : : -: : : : : 6 : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6796978&ulogin=rfidhy : : -: : : : : 3 : : -: : : : : =Yes&types=days&cid=5196539 : : -: : : : : owCamps&ulogin=filterlife-skyboom : : -: : : : : 1 : : -: : : : : 3 : : -: : : : : =Yes&types=days&cid=7010416 : : -: : : : : owCamps : : -: : : : : owClients : : -: : : : 394451&phrasedate=Yes&target_all=1 : owCampStat&cid=4394451&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 12-01-01:2012-11-21,2012-01-01:2012-07-31,2012-03-23:2012-03-23&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : howCampStat&cid=5792419&detail=Yes&types=days : owCampStat&cid=5792419&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-02-14:2013-02-14,&detail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m : : -: : : : : 2=07&d2=02&group=day&goals=0&target_all=yes : : -: : : : : 3 : : -: : : : : 053221&cmd=showCamp : : -: : : : : 6 : : -: : : : : owCamps : : -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -1000 tuples (5m 52s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageVi ews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -: icso : chen : ngin : : : iews : -: urce : gine : eid : : : : -: id : id : : : : : -+======+======+======+====================================================================+=====================================================================+=======+ -: : : : : ype=campdate : : -: : : : type=campdate : : : -: : : : : &mediaType=text : : -: : : : : &payment_type=1 : : -: : : : : d=7 : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0 : : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : ype=campdate : : -: : : : : dirlevel=1372668697.0&cmd=showCamps : : -: : : : : 98.0&authredirlevel=1371874016.0&authredirlevel=1371612009.0&authre : : -: : : : : dirlevel=1370955451.0&cmd=showCamp&cid=5477450 : : -: : : : t : : : -: : : : 1&mediaType=text : : : -: : : : : 9&ulogin=uniparxdir : : -: : : : : owCamps : : -: : : : : 0 : : -: : : : : 41.0&authredirlevel=1373081665.0&authredirlevel=1372822331.0&authre : : -: : : : : dirlevel=1372563095.0&authredirlevel=1372303801.0&authredirlevel=13 : : -: : : : : 72044532.0&authredirlevel=1371730939.0&authredirlevel=1371471570.0& : : -: : : : : authredirlevel=1371212358.0&cmd=showCamp&cid=2856748 : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : -: : : : type=campdate : : : -: : : : : 95.0&authredirlevel=1372303801.0&authredirlevel=1372044532.0&authre : : -: : : : : dirlevel=1371730939.0&authredirlevel=1371471570.0&authredirlevel=13 : : -: : : : : 71212358.0&cmd=showCamp&cid=2856748 : : -: : : : : =225&text_geo=Россия : : -: : : : : 3 : : -: : : : 93 : : : -: : : : : 9 : : -: : : : 19 : : : -: : : : : s=1 : : -: : : : : =1&text_geo=Москва и область : : -: : : : : 8&ulogin=uniparxdir : : -: : : : : 2&ulogin=uniparxdir : : -: : : : : 5&ulogin=uniparxdir : : -: : : : : 6&ulogin=woodenhousedir : : -: : : : : 7&ulogin=uniparxdir : : -: : : : : 4&ulogin=uniparxdir : : -: : : : : 7 : : -: : : : 37 : : : -: : : : 71 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : : 5 : : -: : : : : 5 : : -: : : : 01 : : : -: : : : : 8 : : -: : : : 19 : : : -: : : : : 1 : : -: : : : 65 : : : -: : : : 29 : : : -: : : : 45 : : : -: : : : 03 : : : -: : : : : 0 : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 81 : : : -: : : : 95 : : : -: : : : 40 : : : -: : : : : 5 : : -: : : : 08 : : : -: : : : : 3 : : -: : : : : 1 : : -: : : : : 60.0&authredirlevel=1373047694.0&authredirlevel=1371465714.0&authre : : -: : : : : dirlevel=1370425373.0&authredirlevel=1366055041.0&authredirlevel=13 : : -: : : : : 66054818.0&authredirlevel=1365875147.0&authredirlevel=1365611062.0& : : -: : : : : authredirlevel=1365320979.0&authredirlevel=1365066414.0&cmd=showCam : : -: : : : : p&cid=5516805&ulogin=uniparxdir : : -: : : : : d=7 : : -: : : : : &mediaType=text : : -: : : : edirlevel=1365585299.0, 1365577945.0, 1365574277.0, 1365566946.0, : dirlevel=1365585299.0, 1365577945.0, 1365574277.0, 1365566946.0, 13 : : -: : : : 1365563277.0, 1365559610.0, 1365555942.0, 1365552277.0, 1365544943 : 65563277.0, 1365559610.0, 1365555942.0, 1365552277.0, 1365544943.0, : : -: : : : .0, 1365541249.0, 1365537561.0, 1365533881.0, 1365530189.0, 136552 : 1365541249.0, 1365537561.0, 1365533881.0, 1365530189.0, 1365526529 : : -: : : : 6529.0, 1365522844.0, 1365519158.0, 1365515476.0, 1365511812.0, 13 : .0, 1365522844.0, 1365519158.0, 1365515476.0, 1365511812.0, 1365508 : : -: : : : 65508150.0, 1365504453.0, 1365500842.0, 1365497170.0, 1365493502.0 : 150.0, 1365504453.0, 1365500842.0, 1365497170.0, 1365493502.0, 1365 : : -: : : : , 1365486155.0, 1365482482.0, 1365478812.0, 1365475137.0, 13654714 : 486155.0, 1365482482.0, 1365478812.0, 1365475137.0, 1365471458.0, 1 : : -: : : : 58.0, 1365467780.0, 1365464101.0, 1365460425.0, 1365449525.0, 1365 : 365467780.0, 1365464101.0, 1365460425.0, 1365449525.0, 1365445860.0 : : -: : : : 445860.0, 1365438567.0, 1365434899.0, 1365431229.0, 1365427527.0, : , 1365438567.0, 1365434899.0, 1365431229.0, 1365427527.0, 136542385 : : -: : : : 1365423856.0, 1365420186.0, 1365416498.0, 1365412811.0, 1365409143 : 6.0, 1365420186.0, 1365416498.0, 1365412811.0, 1365409143.0, 136540 : : -: : : : .0, 1365405479.0, 1365401787.0, 1365398118.0, 1365394453.0, 136539 : 5479.0, 1365401787.0, 1365398118.0, 1365394453.0, 1365390789.0, 136 : : -: : : : 0789.0, 1365387124.0, 1365383460.0, 1365379796.0, 1365376129.0, 13 : 5387124.0, 1365383460.0, 1365379796.0, 1365376129.0, 1365372459.0, : : -: : : : 65372459.0, 1365368759.0, 1365365050.0, 1365361381.0, 1365357704.0 : 1365368759.0, 1365365050.0, 1365361381.0, 1365357704.0, 1365350422. : : -: : : : , 1365350422.0, 1365346752.0, 1365343085.0, 1365339417.0, 13653357 : 0, 1365346752.0, 1365343085.0, 1365339417.0, 1365335750.0, 13653284 : : -: : : : 50.0, 1365328485.0, 1365324791.0, 1365320979.0, 1365264347.0, 1365 : 85.0, 1365324791.0, 1365320979.0, 1365264347.0, 1365260692.0&cid=55 : : -: : : : 260692.0&cid=5533477&cmd=showCamp&ulogin=uniparxdir : 33477&cmd=showCamp&ulogin=uniparxdir : : -: : : : : owCamps : : -: : : : 71 : : : -: : : : : 6 : : -: : : : : 3 : : -: : : : : 6 : : -: : : : : owCamps&ulogin=holodilnikru : : -: : : : : dirlevel=1342185655.0&cmd=showCamps&ulogin=beautyland-artha8 : : -: : : : : owCamps : : -: : : : : 3 : : -: : : : : 5 : : -: : : : 43 : : : -: : : : : 5&ulogin=uniparxdir : : -: : : : : 07.0&authredirlevel=1366833461.0&authredirlevel=1366055042.0&authre : : -: : : : : dirlevel=1366054816.0&cmd=showCamp&cid=6419449&ulogin=uniparxdir : : -: : : : : 6 : : -: : : : : 1 : : -: : : : : n= : : -: : : : ks=1 : : : -: : : : howCampStat&cid=5640011&target_type=&detail=yes&y1=13&m1=06&d1=17& : owCampStat&cid=5640011&target_type=&detail=yes&y1=13&m1=06&d1=17&y2 : : -: : : : y2=13&m2=07&d2=28&group=week : =13&m2=07&d2=28&group=day : : -: : : : : 70.0&cmd=showCamp&cid=2753896 : : -: : : : : 8 : : -: : : : : owCamp&cid=7065376#187618038 : : -: : : : : 60.0&authredirlevel=1373047694.0&authredirlevel=1372504061.0&authre : : -: : : : : dirlevel=1371984554.0&cmd=showCamp&cid=5532920&ulogin=uniparxdir : : -: : : : : 6 : : -: : : : 76 : : : -: : : : 78 : : : -: : : : : 8 : : -: : : : : 5 : : -: : : : 55 : : : -: : : : 21 : : : -: : : : : 1 : : -: : : : : =Yes&types=days&cid=6999588 : : -: : : : : owCamps&ulogin=holodilnikru : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : : 0 : : -: : : : howCamps : 16.0&cmd=showCamp&cid=2196876 : : -: : : : : 5 : : -: : : : type=campdate : : : -: : : : : 8 : : -: : : : : 85.0&authredirlevel=1372052820.0&authredirlevel=1371310346.0&authre : : -: : : : : dirlevel=1370938849.0&authredirlevel=1370679627.0&cmd=showCampStat& : : -: : : : : stat_type=campdate : : -: : : : : 69159&phrasedate=Yes&target_all=1 : : -: : : : : 5 : : -: : : : 75 : : : -: : : : : =easy&from=metrika : : -: : : : type=campdate : : : -: : : : : Informers.html : : -: : : : : w.advertising.yandex.ru/welcome/pdf/direct_booklet.pdf : : -: : : : : 7 : : -: : : : ks=1 : : : -: : : : 66&ulogin=merc-traffic : nz_russia/ru/home/van/home/new_vans/special_offers.html?csref=_030_ : : -: : : : : vans:cpc:yandex-direct : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=1475303 : : -: : : : 77 : lshoj-kvartiry : : -: : : : : =Yes&types=days&cid=5995157 : : -: : : : : =Yes&types=days&cid=6883763 : : -: : : : : 7 : : -: : : : d : : : -: : : : : ger : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : : 161272&cmd=showCamp : : -: : : : all : : : -: : : : : 0 : : -: : : : howCamps : 6 : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : 0&ulogin=uniparxdir : : -: : : : : 4 : : -: : : : : 9 : : -: : : : : 05.6&authredirlevel=1372014905.5&authredirlevel=1372014905.4&authre : : -: : : : : dirlevel=1372014905.3&authredirlevel=1372014905.2&authredirlevel=13 : : -: : : : : 72014905.1&authredirlevel=1372014905.0&authredirlevel=1370480737.6& : : -: : : : : authredirlevel=1370480737.5&authredirlevel=1370480737.4&authredirle : : -: : : : : vel=1370480737.3&authredirlevel=1370480737.2&authredirlevel=1370480 : : -: : : : : 737.1&authredirlevel=1370480737.0&cmd=showCampStat&stat_type=campda : : -: : : : : te&target_0=1&target_1=1&target_all=1 : : -: : : : : 6 : : -: : : : l=Yes&types=days&cid=7052441 : owCampStat&cid=7052441&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-26:2013-06-26,2013-06-27:2013-06-27,2013-06-25:2013-06-25&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&tar : : -: : : : : get_all=yes : : -: : : : l=Yes&types=days&cid=7066272 : 66272&phrasedate=Yes&target_all=1 : : -: : : : : 2 : : -: : : : : owCamps&ulogin=visacity08 : : -: : : : : 274687&cmd=showCamp : : -: : : : type=campdate : : : -: : : : l=Yes&types=days&cid=6723854 : owCampStat&cid=6723854&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-26:2013-06-26,2013-06-27:2013-06-27,2013-06-25:2013-06-25&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&tar : : -: : : : : get_all=yes : : -: : : : 42 : : : -: : : : 1&mediaType=text : : : -: : : : : owSubClientCamps : : -: : : : : =Yes&types=days&cid=7066272 : : -: : : : : camps=0&cmd=showCamps&settings_sorted=&sort=name&reverse=0 : : -: : : : : 3 : : -: : : : 56 : owCampStat&cid=5895656&detail=Yes&types=days : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6963923 : : -: : : : : 5 : : -: : : : 09 : : : -: : : : : type=campdate : : -: : : : : ype=campdate : : -: : : : : &mediaType=text&firstbanner=std : : -: : : : howCamp&cid=1971054 : dirlevel=1372767978.0&cmd=showCampStat&cid=1971054&detail=Yes&types : : -: : : : : =days : : -: : : : : owCamps : : -: : : : : 4 : : -: : : : : 5 : : -: : : : : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7134795&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : owSubClientCamps : : -: : : : : =Yes&types=days&cid=6830575 : : -: : : : 916.0&cmd=showCamp&cid=2196876 : 16.0&cid=2196876&cmd=showCamp : : -: : : : &ulogin= : ulogin= : : -: : : : : 8 : : -: : : : : come=yes : : -: : : : : ed : : -: : : : : 6 : : -: : : : : 5 : : -: : : : : 7 : : -: : : : howCamps : 5 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : : 0 : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : : 2 : : -: : : : howCampStat&cid=4588753&offline_stat=0&online_stat=1&stat_periods= : owCamps&mediaType=text&ulogin=FOREX-IMIKS : : -: : : : 2013-06-01:2013-06-30,2013-06-25:2013-07-01,2013-06-17:2013-07-02& : : : -: : : : ulogin=FOREX-IMIKS&stat_type=geo&page=&group=day&y1=2013&m1=06&d1= : : : -: : : : 01&y2=2013&m2=06&d2=30&goals=0&target_all=yes : : : -: : : : : ulogin= : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=3656357 : : -: : : : 10 : : : -: : : : 85 : : : -: : : : : dirlevel=1372758445.0&cmd=showCamp&cid=970693 : : -: : : : : owCamps : : -: : : : site_promotion_search : : : -: : : : t : : : -: : : : 19 : : : -: : : : 58 : : : -: : : : : 43738&stat_type=pages&target_all=1&d1=23&m1=07&y1=12&d2=23&m2=07&y2 : : -: : : : : =13&group=day : : -: : : : : 8 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : howClients : vancedForecast : : -: : : : 41 : : : -: : : : : 32.0&cmd=showCamp&cid=7171725 : : -: : : : 39 : : : -: : : : : 2 : : -: : : : : 6 : : -: : : : : &mediaType=text : : -: : : : : 9 : : -: : : : : 1 : : -: : : : : 6 : : -: : : : : 65759&stat_type=pages&target_all=1&d1=23&m1=07&y1=12&d2=23&m2=07&y2 : : -: : : : : =13&group=day : : -: : : : : 1 : : -: : : : 13 : owCamp&cid=7064513 : : -: : : : : 5 : : -: : : : 08 : : : -: : : : 24 : : : -: : : : : 5 : : -: : : : = : ulogin= : : -: : : : : 6 : : -: : : : : 1 : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=7052441 : owCampStat&cid=7052441&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-26:2013-06-26,2013-06-27:2013-06-27,2013-06-25:2013-06-25&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&tar : : -: : : : : get_all=yes : : -: : : : ransfer : : : -: : : : : 1 : : -: : : : 13 : : : -: : : : : 9 : : -: : : : : 9 : : -: : : : 19 : : : -: : : : : 8 : : -: : : : ncy : : : -: : : : 3243479&tab=media : owCamp&cid=3243479&tab=active : : -: : : : 27 : : : -: : : : howCampStat&cid=3375426&detail=Yes&target_all=1&types=days&show_ba : owCampStat&cid=3375426&detail=Yes&target_all=1&types=days&show_bann : : -: : : : nners_stat=1&page=2 : ers_stat=1&page=1 : : -: : : : : owCamp&cid=7006443&tab=all : : -: : : : : &mediaType=text : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 2 : : -: : : : : 9 : : -: : : : 35 : : : -: : : : 41133;detail=Yes;types=days : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=141133&sho : : -: : : : : w_banners_stat=1 : : -: : : : howCamps : 7 : : -: : : : : 1 : : -: : : : : 0 : : -: : : : howCamps : 5 : : -: : : : 57 : : : -: : : : 77&ulogin=merc-traffic : me_mpc/passengercars/home/servicesandaccessories/special_offers/spr : : -: : : : : ing_2013.html?csref=_mpc_aft_spring-offer2013:cpc:yandex-direct : : -: : : : &ulogin= : : : -: : : : 77 : : : -: : : : howCampStat&cid=2870560&offline_stat=0&online_stat=0&stat_periods= : 0 : : -: : : : 2013-07-01:2013-07-01,2013-06-23:2013-06-24,2013-06-16:2013-06-16& : : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day : : : -: : : : &goals=0&target_all=yes&show_banners_stat=1 : : : -: : : : howCampStat&cid=7022493&detail=Yes&target_all=1&types=days&show_ba : : : -: : : : nners_stat=1&page=1 : : : -: : : : type=campdate : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-18:201 : : -: : : : : 3-07-19,&stat_type=campdate&y1=2013&m1=07&d1=23&y2=2013&m2=07&d2=23 : : -: : : : : &group=day&target_all=1 : : -: : : : : 0 : : -: : : : 20 : : : -: : : : l=Yes&types=days&cid=5826713 : : : -: : : : 999588&phrasedate=Yes&target_all=1 : =Yes&cid=6999588&showCampStat=cmd&types=days&target_all=1 : : -: : : : : ulogin= : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-06-27:2 : : : -: : : : 013-06-28,2013-06-26:2013-06-27,2013-06-01:2013-06-20&stat_type=ca : : : -: : : : mpdate&y1=2013&m1=07&d1=01&y2=2013&m2=07&d2=01&group=day&target_al : : : -: : : : l=1 : : : -: : : : : 6 : : -: : : : : 8 : : -: : : : : 0 : : -: : : : l=Yes&types=days&cid=6932408 : 32408&phrasedate=Yes&target_all=1 : : -: : : : : owCamps&ulogin=holodilnikru : : -: : : : : 98.0&cmd=showCamp&cid=7067235 : : -: : : : : &mediaType=text : : -: : : : : 8 : : -: : : : : 0 : : -: : : : : 1 : : -: : : : : 0 : : -: : : : howCamp&cid=7026131 : : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : 1 : : -: : : : howCampStat&cid=4036889&detail=Yes&types=days : 36889&phrasedate=Yes&target_all=1 : : -: : : : 11 : 13411=1#list : : -: : : : 12 : : : -: : : : howCamp&cid=4246248&tab=decline : owCamp&cid=4246248&tab=active : : -: : : : : 9 : : -: : : : : 6 : : -: : : : 07 : : : -: : : : l=Yes&types=days&cid=5033617 : : : -: : : : : 0 : : -: : : : : 1 : : -: : : : : 4 : : -: : : : 1&mediaType=text : : : -: : : : : 2 : : -: : : : : 7 : : -: : : : l=Yes&types=days&cid=7034128 : : : -: : : : 99 : : : -: : : : id=7 : : : -: : : : : 8 : : -: : : : : 7 : : -: : : : : owCamps&ulogin=visamaster08 : : -: : : : 43 : : : -: : : : howCamps&ulogin=dc12a : owCampStat&cid=5658570&cid=5753133&cid=5701003&stat_type=campdate&u : : -: : : : : login=dc12a&csrf_token=shocsLKxtAnjy3K1 : : -: : : : : 0 : : -: : : : : owCamps&ulogin=TENDER-SELDON-IMIKS : : -: : : : : 73.0&cmd=showCampStat&cid=6421801&cid=6424294&cid=6487283&cid=64938 : : -: : : : : 29&cid=6494462&cid=6495189&stat_type=campdate&csrf_token=pRFezf5GIV : : -: : : : : r0wz8J : : -: : : : 77 : : : -: : : : : =Yes&types=days&cid=4537562 : : -: : : : : =Yes&types=days&cid=3353427 : : -: : : : 01 : : : -: : : : : 7 : : -: : : : : 2 : : -: : : : 52 : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6932408 : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=7162500 : : : -: : : : howCampStat&cid=1714344&offline_stat=0&online_stat=0&stat_periods= : owCampStat&cid=1714344&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : 2013-06-01:2013-06-30,2013-06-28:2013-06-29,2013-06-27:2013-06-28& : 13-06-01:2013-06-30,2013-06-28:2013-06-29,2013-06-27:2013-06-28&det : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&goa : : -: : : : &goals=0&target_all=yes&show_banners_stat=1 : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : : vancedForecast : : -: : : : : 3 : : -: : : : : 4 : : -: : : : 66 : : : -: : : : : 1 : : -: : : : howCampStat&cid=5360801&offline_stat=0&online_stat=1&stat_periods= : : : -: : : : &types=total&page=&group=day&y1=2012&m1=07&d1=02&y2=2013&m2=07&d2= : : : -: : : : 02 : : : -: : : : : 5 : : -: : : : t : mpleCamp : : -: : : : : 5 : : -: : : : : 2 : : -: : : : : 5 : : -: : : : : 6 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 4 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=7032884 : : -: : : : ts : : : -: : : : howCamp&cid=6411626&tab=off : owCamp&cid=6411626 : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : : 3 : : -: : : : : 6 : : -: : : : : 5 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6844274 : : -: : : : l=Yes&cid=2139196&ulogin=onclinic-sm&showCampStat=cmd&types=days&t : owCampStat&cid=2139196&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : arget_all=1&d1=01&m1=06&y1=13&d2=01&m2=07&y2=13&group=day : 13-06-01:2013-07-02,2013-06-21:2013-07-02,2013-06-24:2013-07-02&ulo : : -: : : : : gin=onclinic-sm&detail=yes&page=&y1=2013&m1=06&d1=01&y2=2013&m2=07& : : -: : : : : d2=02&group=day&goals=0&target_all=yes : : -: : : : : 6 : : -: : : : : 1 : : -: : : : 75 : : : -: : : : : 6 : : -: : : : 148542&stat_type=pages&target_all=1 : =Yes&cid=7148542&showCampStat=cmd&types=days&target_all=1 : : -: : : : : owCampStat&cid=4917291&detail=Yes&types=days : : -: : : : : 1 : : -: : : : 7-.31620-1.1372703844295129.198343986.198343986..53121502.1406.512 : 7 : : -: : : : l=Yes&types=days&cid=7041629 : : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6027971 : : -: : : : 20 : : : -: : : : : vancedForecast : : -: : : : : =Yes&types=days&cid=7034128 : : -: : : : =total&target_all=1&page=1&cmd=showCampStat&cid=3016175&show_banne : owCampStat&cid=3016175&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : rs_stat=1 : 12-02-03:2012-02-04,2011-08-01:2011-09-04,2010-12-29:2011-01-02&typ : : -: : : : : es=total&page=&group=day&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 13411=1#list : : -: : : : l=Yes&types=days&cid=88689 : : : -: : : : : 88.0 : : -: : : : howCamps&mediaType=text&ulogin=super-svictor : ype=campdate&&ulogin=super-svictor : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=6940624 : : -: : : : : &mediaType=text&firstbanner=easy : : -: : : : : dirlevel=1372663385.0&cmd=showCamp&cid=2139196&ulogin=onclinic-sm : : -: : : : 97&ulogin=mirkli-webmart : : : -: : : : : 6 : : -: : : : : 0 : : -: : : : l=Yes&types=days&cid=4644721 : owCampStat&cid=4644721&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-05-27:2013-05-28,2011-09-12:2011-09-12,2011-06-05:2011-07-05&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 5 : : -: : : : : dirlevel=1372412588.0&authredirlevel=1372408461.0&authredirlevel=13 : : -: : : : : 72403851.0&authredirlevel=1372339893.0&authredirlevel=1372335674.0& : : -: : : : : authredirlevel=1372331762.0&authredirlevel=1372323148.0&authredirle : : -: : : : : vel=1372313398.0&authredirlevel=1372309630.0&authredirlevel=1372252 : : -: : : : : 641.0&authredirlevel=1372248697.0&authredirlevel=1372244608.0&authr : : -: : : : : edirlevel=1372240548.0&authredirlevel=1372231517.0&authredirlevel=1 : : -: : : : : 372227248.0&authredirlevel=1372074352.0&authredirlevel=1371476832.0 : : -: : : : : &cmd=showCamp&tab=active&cid=6051926 : : -: : : : howCamp&cid=6501536 : : : -: : : : howCampStat&cid=7022493&detail=Yes&target_all=1&types=days&show_ba : : : -: : : : nners_stat=1&page=2 : : : -: : : : : &mediaType=text : : -: : : : : =Yes&types=days&cid=7041629 : : -: : : : : 9 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=2704830 : : : -: : : : : =Yes&types=days&cid=6100946 : : -: : : : : 0 : : -: : : : : 8 : : -: : : : : 37.0&authredirlevel=1371612009.0&authredirlevel=1370955451.0&cmd=sh : : -: : : : : owCamp&cid=5477450 : : -: : : : 18 : : : -: : : : 17 : : : -: : : : : 0 : : -: : : : : 6 : : -: : : : : dirlevel=1374312836.0&cmd=showCamps : : -: : : : : 5 : : -: : : : 57 : : : -: : : : : 5 : : -: : : : : 3 : : -: : : : l=Yes&cid=2870560&showCampStat=cmd&types=days&target_all=1&d1=02&m : 0 : : -: : : : 1=07&y1=13&d2=02&m2=07&y2=13&group=day : : : -: : : : : 0 : : -: : : : : 3 : : -: : : : &ulogin= : ulogin= : : -: : : : 12 : : : -: : : : oInformers.html : : : -: : : : howCamp&cid=4693423&tab=active : owCamp&cid=4693423&tab=wait : : -: : : : : 2 : : -: : : : : 1 : : -: : : : : 9 : : -: : : : : 11.0&authredirlevel=1372755166.0&cmd=showClients : : -: : : : : 1 : : -: : : : : 1 : : -: : : : : 3 : : -: : : : : 066835&cmd=showCamp : : -: : : : : 0 : : -: : : : 6 : : : -: : : : : 3 : : -: : : : : 7 : : -: : : : howCamp&cid=970693 : owCamp&cid=2997085 : : -: : : : : 8 : : -: : : : 1&mediaType=text : : : -: : : : : 6 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : 79 : owCamp&cid=7108679&tab=&page=2 : : -: : : : : =Yes&types=days&cid=7169384 : : -: : : : oInformers.html : : : -: : : : : dirlevel=1372772163.0&cmd=showCamps&tab=all&ulogin=spb-tester1 : : -: : : : : 8 : : -: : : : : 0 : : -: : : : 61 : : : -: : : : : vancedForecast : : -: : : : : =Yes&types=days&cid=6974488 : : -: : : : 84 : : : -: : : : : owCamp&cid=6833286 : : -: : : : : 5 : : -: : : : : 4 : : -: : : : howCamps&ulogin=major-auto-RW : ype=campdate&&ulogin=major-auto-RW : : -: : : : : 9 : : -: : : : ditBannerEasy&cid=6939766&bid=178815009 : dirlevel=1372759245.0&&cid=6939766&cmd=showCamp : : -: : : : ned : : : -: : : : ditCamp&cid=2870560 : owCamp&cid=2870560 : : -: : : : : 8 : : -: : : : howCampStat&cid=7058631&detail=Yes&types=days : 58631&stat_type=pages&target_all=1 : : -: : : : : =Yes&types=days&cid=3714884 : : -: : : : 97 : : : -: : : : : 0 : : -: : : : bids=5756034&cid=1840733 : : : -: : : : : =Yes&types=days&cid=4641928 : : -: : : : 91 : : : -: : : : : 9 : : -: : : : : 3 : : -: : : : : 49.0&authredirlevel=1372754333.0&authredirlevel=1372750347.0&cmd=sh : : -: : : : : owCamp&cid=5694785&ulogin=otzyvavto : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0#6136521 : : -: : : : l=Yes&types=days&cid=7153732 : 53732&phrasedate=Yes&target_all=1 : : -: : : : : 4 : : -: : : : : 1 : : -: : : : : 1 : : -: : : : : =213&text_geo=Москва : : -: : : : : 0 : : -: : : : : 6 : : -: : : : : ype=campdate : : -: : : : : 3 : : -: : : : : &mediaType=text&firstbanner=std : : -: : : : : 4 : : -: : : : : 8 : : -: : : : 92 : owCampStat&cid=6486492&detail=Yes&types=days : : -: : : : l=Yes&types=days&cid=6975816 : 6 : : -: : : : : owCamps : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 1 : : -: : : : 75 : : : -: : : : : =Yes&types=days&cid=6942376 : : -: : : : 68 : : : -: : : : : 7 : : -: : : : : 4 : : -: : : : : 9 : : -: : : : l=Yes&types=days&cid=6851682 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=6851682&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : =Yes&types=days&cid=2704830 : : -: : : : 21 : : : -: : : : 04 : owCampStat&cid=7057804&detail=Yes&types=days : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=6486629 : : -: : : : 73 : : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 31 : owCamp&cid=6839731 : : -: : : : 066272&phrasedate=Yes&target_all=1 : : : -: : : : howCampStat&cid=2803428&offline_stat=0&online_stat=0&stat_periods= : 8 : : -: : : : 2013-06-01:2013-06-24,2013-06-01:2013-06-14,2013-06-03:2013-06-03& : : : -: : : : phrasedate=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group : : : -: : : : =day&goals=0&target_all=yes : : : -: : : : : 5 : : -: : : : : 86.0&authredirlevel=1372751903.0&cmd=showCamp&cid=3055906 : : -: : : : : 3 : : -: : : : : 0 : : -: : : : : =Yes&types=days&cid=6157316 : : -: : : : : =Yes&types=days&cid=3502476 : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0#5809640 : : -: : : : : =Yes&types=days&cid=6860824 : : -: : : : edirlevel=1372746442.0&authredirlevel=1372714097.0&authredirlevel= : owCampStat&cid=3226358&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : 1372698041.0&authredirlevel=1372693736.0&cmd=showCampStat&cid=3226 : 13-06-24:2013-07-02,&phrasedate=yes&page=&y1=2013&m1=06&d1=24&y2=20 : : -: : : : 358&offline_stat=0&online_stat=0&stat_periods=&phrasedate=yes&page : 13&m2=07&d2=02&group=day&goals=0&target_all=yes : : -: : : : =&y1=2013&m1=06&d1=24&y2=2013&m2=07&d2=01&group=day&goals=0&target : : : -: : : : _all=yes : : : -: : : : : 5 : : -: : : : howCamps&ulogin=dc12a : owCampStat&cid=7041500&stat_type=campdate&ulogin=dc12a&csrf_token=Q : : -: : : : : un6g1847J4TOMJW : : -: : : : howCamp&tab=all&cid=6831216 : owCamp&cid=6831216&tab=active : : -: : : : : 3 : : -: : : : : 064001&cmd=showCamp : : -: : : : : =Yes&types=days&cid=7165103 : : -: : : : : 6 : : -: : : : : ype=campdate : : -: : : : 04 : : : -: : : : 622777;offline_stat=0;online_stat=1;stat_periods=2013-06-01:2013-0 : : : -: : : : 7-02,2013-04-01:2013-06-28,2013-03-25:2013-06-28;detail=yes;page=; : : : -: : : : y1=2013;m1=06;d1=01;y2=2013;m2=07;d2=02;group=day;goals=0;target_a : : : -: : : : ll=yes : : : -: : : : : =Yes&types=days&cid=7066904 : : -: : : : : 90.0&cmd=showCamp&cid=7142178 : : -: : : : : =Yes&types=days&cid=4917905 : : -: : : : 171931&phrasedate=Yes&target_all=1 : : : -: : : : : 9 : : -: : : : l=Yes&types=days&cid=5967378 : owCampStat&cid=5967378&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-18:2013-07-02,2013-06-18:2013-06-24,2013-06-24:2013-06-30&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&goa : : -: : : : : ls=0&target_all=yes : : -: : : : : =Yes&types=days&cid=5685110 : : -: : : : : 3 : : -: : : : : owCamp&cid=7163322#197090829 : : -: : : : : 1 : : -: : : : l=Yes&types=days&cid=7087520 : : : -: : : : : =Yes&types=days&cid=7064245 : : -: : : : 36 : : : -: : : : : 4 : : -: : : : : owCamp&cid=2217231 : : -: : : : : 064001&cmd=showCamp : : -: : : : : 0 : : -: : : : : 2 : : -: : : : : 4 : : -: : : : : 0 : : -: : : : : 9 : : -: : : : 68 : : : -: : : : howCamp&cid=2838772&tab=off : owCamp&cid=2838772&tab=active : : -: : : : : 8 : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=7036399 : : : -: : : : 73 : : : -: : : : howCamp&cid=6155736#165459179 : : : -: : : : : 5 : : -: : : : : 2 : : -: : : : &ulogin= : : : -: : : : : 1 : : -: : : : : 0 : : -: : : : : 8 : : -: : : : 61 : : : -: : : : l=Yes&types=days&cid=7051349 : : : -: : : : : =Yes&types=days&cid=7065579 : : -: : : : : 5 : : -: : : : 02 : : : -: : : : : 27.0&cmd=showCamp&cid=4670220 : : -: : : : : 9 : : -: : : : : 1 : : -: : : : : =Yes&types=days&cid=4764350 : : -: : : : : 6 : : -: : : : : 3 : : -: : : : 35 : owCampStat&cid=6921735&detail=Yes&types=days : : -: : : : : =Yes&types=days&cid=6851682 : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : : =Yes&types=days&cid=7055483 : : -: : : : 14 : : : -: : : : : 4 : : -: : : : 69 : : : -: : : : : 4 : : -: : : : : campdate : : -: : : : : 2 : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : ditCamp&cid=7100989 : 00989;offline_stat=0;online_stat=1;stat_periods=2013-01-10:2013-01- : : -: : : : : 28,2013-01-10:2013-01-27,2013-01-10:2013-01-26;detail=yes;page=;y1= : : -: : : : : 2013;m1=06;d1=22;y2=2013;m2=07;d2=22;group=day;goals=0;target_all=y : : -: : : : : es : : -: : : : : 2 : : -: : : : : 4 : : -: : : : 35 : : : -: : : : : 7 : : -: : : : : g14AZEA.pl?authredirlevel=1372773289.0&cmd=showCamps&ulogin=eslucha : : -: : : : : k : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : howCamps : 7 : : -: : : : 92 : : : -: : : : : 02.0&cmd=showClients : : -: : : : : 1 : : -: : : : howCamp&cid=7090606 : owCampStat&cid=7090606&detail=Yes&types=days : : -: : : : 61 : : : -: : : : : =Yes&types=days&cid=6994484 : : -: : : : : 3 : : -: : : : : 4 : : -: : : : : 6 : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6796978&ulogin=rfidhy : : -: : : : : 3 : : -: : : : : =Yes&types=days&cid=5196539 : : -: : : : : owCamps&ulogin=filterlife-skyboom : : -: : : : : 1 : : -: : : : : 3 : : -: : : : : =Yes&types=days&cid=7010416 : : -: : : : : owCamps : : -: : : : : owClients : : -: : : : 394451&phrasedate=Yes&target_all=1 : owCampStat&cid=4394451&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 12-01-01:2012-11-21,2012-01-01:2012-07-31,2012-03-23:2012-03-23&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : howCampStat&cid=5792419&detail=Yes&types=days : owCampStat&cid=5792419&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-02-14:2013-02-14,&detail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m : : -: : : : : 2=07&d2=02&group=day&goals=0&target_all=yes : : -: : : : : 3 : : -: : : : : 053221&cmd=showCamp : : -: : : : : 6 : : -: : : : : owCamps : : -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -1000 tuples (5m 36s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageVi ews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -: icso : chen : ngin : : : iews : -: urce : gine : eid : : : : -: id : id : : : : : -+======+======+======+====================================================================+=====================================================================+=======+ -: : : : : ype=campdate : : -: : : : type=campdate : : : -: : : : : &mediaType=text : : -: : : : : &payment_type=1 : : -: : : : : d=7 : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0 : : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : ype=campdate : : -: : : : : dirlevel=1372668697.0&cmd=showCamps : : -: : : : : 98.0&authredirlevel=1371874016.0&authredirlevel=1371612009.0&authre : : -: : : : : dirlevel=1370955451.0&cmd=showCamp&cid=5477450 : : -: : : : t : : : -: : : : 1&mediaType=text : : : -: : : : : 9&ulogin=uniparxdir : : -: : : : : owCamps : : -: : : : : 0 : : -: : : : : 41.0&authredirlevel=1373081665.0&authredirlevel=1372822331.0&authre : : -: : : : : dirlevel=1372563095.0&authredirlevel=1372303801.0&authredirlevel=13 : : -: : : : : 72044532.0&authredirlevel=1371730939.0&authredirlevel=1371471570.0& : : -: : : : : authredirlevel=1371212358.0&cmd=showCamp&cid=2856748 : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : -: : : : type=campdate : : : -: : : : : 95.0&authredirlevel=1372303801.0&authredirlevel=1372044532.0&authre : : -: : : : : dirlevel=1371730939.0&authredirlevel=1371471570.0&authredirlevel=13 : : -: : : : : 71212358.0&cmd=showCamp&cid=2856748 : : -: : : : : =225&text_geo=Россия : : -: : : : : 3 : : -: : : : 93 : : : -: : : : : 9 : : -: : : : 19 : : : -: : : : : s=1 : : -: : : : : =1&text_geo=Москва и область : : -: : : : : 8&ulogin=uniparxdir : : -: : : : : 2&ulogin=uniparxdir : : -: : : : : 5&ulogin=uniparxdir : : -: : : : : 6&ulogin=woodenhousedir : : -: : : : : 7&ulogin=uniparxdir : : -: : : : : 4&ulogin=uniparxdir : : -: : : : : 7 : : -: : : : 37 : : : -: : : : 71 : : : -: : : : : 9 : : -: : : : : 1 : : -: : : : : 5 : : -: : : : : 5 : : -: : : : 01 : : : -: : : : : 8 : : -: : : : 19 : : : -: : : : : 1 : : -: : : : 65 : : : -: : : : 29 : : : -: : : : 45 : : : -: : : : 03 : : : -: : : : : 0 : : -: : : : : 9 : : -: : : : : 1 : : -: : : : 81 : : : -: : : : 95 : : : -: : : : 40 : : : -: : : : : 5 : : -: : : : 08 : : : -: : : : : 3 : : -: : : : : 1 : : -: : : : : 60.0&authredirlevel=1373047694.0&authredirlevel=1371465714.0&authre : : -: : : : : dirlevel=1370425373.0&authredirlevel=1366055041.0&authredirlevel=13 : : -: : : : : 66054818.0&authredirlevel=1365875147.0&authredirlevel=1365611062.0& : : -: : : : : authredirlevel=1365320979.0&authredirlevel=1365066414.0&cmd=showCam : : -: : : : : p&cid=5516805&ulogin=uniparxdir : : -: : : : : d=7 : : -: : : : : &mediaType=text : : -: : : : edirlevel=1365585299.0, 1365577945.0, 1365574277.0, 1365566946.0, : dirlevel=1365585299.0, 1365577945.0, 1365574277.0, 1365566946.0, 13 : : -: : : : 1365563277.0, 1365559610.0, 1365555942.0, 1365552277.0, 1365544943 : 65563277.0, 1365559610.0, 1365555942.0, 1365552277.0, 1365544943.0, : : -: : : : .0, 1365541249.0, 1365537561.0, 1365533881.0, 1365530189.0, 136552 : 1365541249.0, 1365537561.0, 1365533881.0, 1365530189.0, 1365526529 : : -: : : : 6529.0, 1365522844.0, 1365519158.0, 1365515476.0, 1365511812.0, 13 : .0, 1365522844.0, 1365519158.0, 1365515476.0, 1365511812.0, 1365508 : : -: : : : 65508150.0, 1365504453.0, 1365500842.0, 1365497170.0, 1365493502.0 : 150.0, 1365504453.0, 1365500842.0, 1365497170.0, 1365493502.0, 1365 : : -: : : : , 1365486155.0, 1365482482.0, 1365478812.0, 1365475137.0, 13654714 : 486155.0, 1365482482.0, 1365478812.0, 1365475137.0, 1365471458.0, 1 : : -: : : : 58.0, 1365467780.0, 1365464101.0, 1365460425.0, 1365449525.0, 1365 : 365467780.0, 1365464101.0, 1365460425.0, 1365449525.0, 1365445860.0 : : -: : : : 445860.0, 1365438567.0, 1365434899.0, 1365431229.0, 1365427527.0, : , 1365438567.0, 1365434899.0, 1365431229.0, 1365427527.0, 136542385 : : -: : : : 1365423856.0, 1365420186.0, 1365416498.0, 1365412811.0, 1365409143 : 6.0, 1365420186.0, 1365416498.0, 1365412811.0, 1365409143.0, 136540 : : -: : : : .0, 1365405479.0, 1365401787.0, 1365398118.0, 1365394453.0, 136539 : 5479.0, 1365401787.0, 1365398118.0, 1365394453.0, 1365390789.0, 136 : : -: : : : 0789.0, 1365387124.0, 1365383460.0, 1365379796.0, 1365376129.0, 13 : 5387124.0, 1365383460.0, 1365379796.0, 1365376129.0, 1365372459.0, : : -: : : : 65372459.0, 1365368759.0, 1365365050.0, 1365361381.0, 1365357704.0 : 1365368759.0, 1365365050.0, 1365361381.0, 1365357704.0, 1365350422. : : -: : : : , 1365350422.0, 1365346752.0, 1365343085.0, 1365339417.0, 13653357 : 0, 1365346752.0, 1365343085.0, 1365339417.0, 1365335750.0, 13653284 : : -: : : : 50.0, 1365328485.0, 1365324791.0, 1365320979.0, 1365264347.0, 1365 : 85.0, 1365324791.0, 1365320979.0, 1365264347.0, 1365260692.0&cid=55 : : -: : : : 260692.0&cid=5533477&cmd=showCamp&ulogin=uniparxdir : 33477&cmd=showCamp&ulogin=uniparxdir : : -: : : : : owCamps : : -: : : : 71 : : : -: : : : : 6 : : -: : : : : 3 : : -: : : : : 6 : : -: : : : : owCamps&ulogin=holodilnikru : : -: : : : : dirlevel=1342185655.0&cmd=showCamps&ulogin=beautyland-artha8 : : -: : : : : owCamps : : -: : : : : 3 : : -: : : : : 5 : : -: : : : 43 : : : -: : : : : 5&ulogin=uniparxdir : : -: : : : : 07.0&authredirlevel=1366833461.0&authredirlevel=1366055042.0&authre : : -: : : : : dirlevel=1366054816.0&cmd=showCamp&cid=6419449&ulogin=uniparxdir : : -: : : : : 6 : : -: : : : : 1 : : -: : : : : n= : : -: : : : ks=1 : : : -: : : : howCampStat&cid=5640011&target_type=&detail=yes&y1=13&m1=06&d1=17& : owCampStat&cid=5640011&target_type=&detail=yes&y1=13&m1=06&d1=17&y2 : : -: : : : y2=13&m2=07&d2=28&group=week : =13&m2=07&d2=28&group=day : : -: : : : : 70.0&cmd=showCamp&cid=2753896 : : -: : : : : 8 : : -: : : : : owCamp&cid=7065376#187618038 : : -: : : : : 60.0&authredirlevel=1373047694.0&authredirlevel=1372504061.0&authre : : -: : : : : dirlevel=1371984554.0&cmd=showCamp&cid=5532920&ulogin=uniparxdir : : -: : : : : 6 : : -: : : : 76 : : : -: : : : 78 : : : -: : : : : 8 : : -: : : : : 5 : : -: : : : 55 : : : -: : : : 21 : : : -: : : : : 1 : : -: : : : : =Yes&types=days&cid=6999588 : : -: : : : : owCamps&ulogin=holodilnikru : : -: : : : =addtime&reverse=1&ws_place=0&ws_time=1d&ws_cid=0&ws_done=0 : : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : : 0 : : -: : : : howCamps : 16.0&cmd=showCamp&cid=2196876 : : -: : : : : 5 : : -: : : : type=campdate : : : -: : : : : 8 : : -: : : : : 85.0&authredirlevel=1372052820.0&authredirlevel=1371310346.0&authre : : -: : : : : dirlevel=1370938849.0&authredirlevel=1370679627.0&cmd=showCampStat& : : -: : : : : stat_type=campdate : : -: : : : : 69159&phrasedate=Yes&target_all=1 : : -: : : : : 5 : : -: : : : 75 : : : -: : : : : =easy&from=metrika : : -: : : : type=campdate : : : -: : : : : Informers.html : : -: : : : : w.advertising.yandex.ru/welcome/pdf/direct_booklet.pdf : : -: : : : : 7 : : -: : : : ks=1 : : : -: : : : 66&ulogin=merc-traffic : nz_russia/ru/home/van/home/new_vans/special_offers.html?csref=_030_ : : -: : : : : vans:cpc:yandex-direct : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=1475303 : : -: : : : 77 : lshoj-kvartiry : : -: : : : : =Yes&types=days&cid=5995157 : : -: : : : : =Yes&types=days&cid=6883763 : : -: : : : : 7 : : -: : : : d : : : -: : : : : ger : : -: : : : 1&mediaType=text&from=fp_bbb : : : -: : : : : 161272&cmd=showCamp : : -: : : : all : : : -: : : : : 0 : : -: : : : howCamps : 6 : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : 0&ulogin=uniparxdir : : -: : : : : 4 : : -: : : : : 9 : : -: : : : : 05.6&authredirlevel=1372014905.5&authredirlevel=1372014905.4&authre : : -: : : : : dirlevel=1372014905.3&authredirlevel=1372014905.2&authredirlevel=13 : : -: : : : : 72014905.1&authredirlevel=1372014905.0&authredirlevel=1370480737.6& : : -: : : : : authredirlevel=1370480737.5&authredirlevel=1370480737.4&authredirle : : -: : : : : vel=1370480737.3&authredirlevel=1370480737.2&authredirlevel=1370480 : : -: : : : : 737.1&authredirlevel=1370480737.0&cmd=showCampStat&stat_type=campda : : -: : : : : te&target_0=1&target_1=1&target_all=1 : : -: : : : : 6 : : -: : : : l=Yes&types=days&cid=7052441 : owCampStat&cid=7052441&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-26:2013-06-26,2013-06-27:2013-06-27,2013-06-25:2013-06-25&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&tar : : -: : : : : get_all=yes : : -: : : : l=Yes&types=days&cid=7066272 : 66272&phrasedate=Yes&target_all=1 : : -: : : : : 2 : : -: : : : : owCamps&ulogin=visacity08 : : -: : : : : 274687&cmd=showCamp : : -: : : : type=campdate : : : -: : : : l=Yes&types=days&cid=6723854 : owCampStat&cid=6723854&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-26:2013-06-26,2013-06-27:2013-06-27,2013-06-25:2013-06-25&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&tar : : -: : : : : get_all=yes : : -: : : : 42 : : : -: : : : 1&mediaType=text : : : -: : : : : owSubClientCamps : : -: : : : : =Yes&types=days&cid=7066272 : : -: : : : : camps=0&cmd=showCamps&settings_sorted=&sort=name&reverse=0 : : -: : : : : 3 : : -: : : : 56 : owCampStat&cid=5895656&detail=Yes&types=days : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6963923 : : -: : : : : 5 : : -: : : : 09 : : : -: : : : : type=campdate : : -: : : : : ype=campdate : : -: : : : : &mediaType=text&firstbanner=std : : -: : : : howCamp&cid=1971054 : dirlevel=1372767978.0&cmd=showCampStat&cid=1971054&detail=Yes&types : : -: : : : : =days : : -: : : : : owCamps : : -: : : : : 4 : : -: : : : : 5 : : -: : : : : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=7134795&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : owSubClientCamps : : -: : : : : =Yes&types=days&cid=6830575 : : -: : : : 916.0&cmd=showCamp&cid=2196876 : 16.0&cid=2196876&cmd=showCamp : : -: : : : &ulogin= : ulogin= : : -: : : : : 8 : : -: : : : : come=yes : : -: : : : : ed : : -: : : : : 6 : : -: : : : : 5 : : -: : : : : 7 : : -: : : : howCamps : 5 : : -: : : : : 1 : : -: : : : : 2 : : -: : : : : 0 : : -: : : : : =Yes&types=days&cid=7100989 : : -: : : : : 2 : : -: : : : howCampStat&cid=4588753&offline_stat=0&online_stat=1&stat_periods= : owCamps&mediaType=text&ulogin=FOREX-IMIKS : : -: : : : 2013-06-01:2013-06-30,2013-06-25:2013-07-01,2013-06-17:2013-07-02& : : : -: : : : ulogin=FOREX-IMIKS&stat_type=geo&page=&group=day&y1=2013&m1=06&d1= : : : -: : : : 01&y2=2013&m2=06&d2=30&goals=0&target_all=yes : : : -: : : : : ulogin= : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=3656357 : : -: : : : 10 : : : -: : : : 85 : : : -: : : : : dirlevel=1372758445.0&cmd=showCamp&cid=970693 : : -: : : : : owCamps : : -: : : : site_promotion_search : : : -: : : : t : : : -: : : : 19 : : : -: : : : 58 : : : -: : : : : 43738&stat_type=pages&target_all=1&d1=23&m1=07&y1=12&d2=23&m2=07&y2 : : -: : : : : =13&group=day : : -: : : : : 8 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : howClients : vancedForecast : : -: : : : 41 : : : -: : : : : 32.0&cmd=showCamp&cid=7171725 : : -: : : : 39 : : : -: : : : : 2 : : -: : : : : 6 : : -: : : : : &mediaType=text : : -: : : : : 9 : : -: : : : : 1 : : -: : : : : 6 : : -: : : : : 65759&stat_type=pages&target_all=1&d1=23&m1=07&y1=12&d2=23&m2=07&y2 : : -: : : : : =13&group=day : : -: : : : : 1 : : -: : : : 13 : owCamp&cid=7064513 : : -: : : : : 5 : : -: : : : 08 : : : -: : : : 24 : : : -: : : : : 5 : : -: : : : = : ulogin= : : -: : : : : 6 : : -: : : : : 1 : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=7052441 : owCampStat&cid=7052441&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-26:2013-06-26,2013-06-27:2013-06-27,2013-06-25:2013-06-25&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&tar : : -: : : : : get_all=yes : : -: : : : ransfer : : : -: : : : : 1 : : -: : : : 13 : : : -: : : : : 9 : : -: : : : : 9 : : -: : : : 19 : : : -: : : : : 8 : : -: : : : ncy : : : -: : : : 3243479&tab=media : owCamp&cid=3243479&tab=active : : -: : : : 27 : : : -: : : : howCampStat&cid=3375426&detail=Yes&target_all=1&types=days&show_ba : owCampStat&cid=3375426&detail=Yes&target_all=1&types=days&show_bann : : -: : : : nners_stat=1&page=2 : ers_stat=1&page=1 : : -: : : : : owCamp&cid=7006443&tab=all : : -: : : : : &mediaType=text : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 2 : : -: : : : : 9 : : -: : : : 35 : : : -: : : : 41133;detail=Yes;types=days : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=141133&sho : : -: : : : : w_banners_stat=1 : : -: : : : howCamps : 7 : : -: : : : : 1 : : -: : : : : 0 : : -: : : : howCamps : 5 : : -: : : : 57 : : : -: : : : 77&ulogin=merc-traffic : me_mpc/passengercars/home/servicesandaccessories/special_offers/spr : : -: : : : : ing_2013.html?csref=_mpc_aft_spring-offer2013:cpc:yandex-direct : : -: : : : &ulogin= : : : -: : : : 77 : : : -: : : : howCampStat&cid=2870560&offline_stat=0&online_stat=0&stat_periods= : 0 : : -: : : : 2013-07-01:2013-07-01,2013-06-23:2013-06-24,2013-06-16:2013-06-16& : : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day : : : -: : : : &goals=0&target_all=yes&show_banners_stat=1 : : : -: : : : howCampStat&cid=7022493&detail=Yes&target_all=1&types=days&show_ba : : : -: : : : nners_stat=1&page=1 : : : -: : : : type=campdate : owCampStat&offline_stat=0&online_stat=0&stat_periods=2013-07-18:201 : : -: : : : : 3-07-19,&stat_type=campdate&y1=2013&m1=07&d1=23&y2=2013&m2=07&d2=23 : : -: : : : : &group=day&target_all=1 : : -: : : : : 0 : : -: : : : 20 : : : -: : : : l=Yes&types=days&cid=5826713 : : : -: : : : 999588&phrasedate=Yes&target_all=1 : =Yes&cid=6999588&showCampStat=cmd&types=days&target_all=1 : : -: : : : : ulogin= : : -: : : : howCampStat&offline_stat=0&online_stat=0&stat_periods=2013-06-27:2 : : : -: : : : 013-06-28,2013-06-26:2013-06-27,2013-06-01:2013-06-20&stat_type=ca : : : -: : : : mpdate&y1=2013&m1=07&d1=01&y2=2013&m2=07&d2=01&group=day&target_al : : : -: : : : l=1 : : : -: : : : : 6 : : -: : : : : 8 : : -: : : : : 0 : : -: : : : l=Yes&types=days&cid=6932408 : 32408&phrasedate=Yes&target_all=1 : : -: : : : : owCamps&ulogin=holodilnikru : : -: : : : : 98.0&cmd=showCamp&cid=7067235 : : -: : : : : &mediaType=text : : -: : : : : 8 : : -: : : : : 0 : : -: : : : : 1 : : -: : : : : 0 : : -: : : : howCamp&cid=7026131 : : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : : 1 : : -: : : : howCampStat&cid=4036889&detail=Yes&types=days : 36889&phrasedate=Yes&target_all=1 : : -: : : : 11 : 13411=1#list : : -: : : : 12 : : : -: : : : howCamp&cid=4246248&tab=decline : owCamp&cid=4246248&tab=active : : -: : : : : 9 : : -: : : : : 6 : : -: : : : 07 : : : -: : : : l=Yes&types=days&cid=5033617 : : : -: : : : : 0 : : -: : : : : 1 : : -: : : : : 4 : : -: : : : 1&mediaType=text : : : -: : : : : 2 : : -: : : : : 7 : : -: : : : l=Yes&types=days&cid=7034128 : : : -: : : : 99 : : : -: : : : id=7 : : : -: : : : : 8 : : -: : : : : 7 : : -: : : : : owCamps&ulogin=visamaster08 : : -: : : : 43 : : : -: : : : howCamps&ulogin=dc12a : owCampStat&cid=5658570&cid=5753133&cid=5701003&stat_type=campdate&u : : -: : : : : login=dc12a&csrf_token=shocsLKxtAnjy3K1 : : -: : : : : 0 : : -: : : : : owCamps&ulogin=TENDER-SELDON-IMIKS : : -: : : : : 73.0&cmd=showCampStat&cid=6421801&cid=6424294&cid=6487283&cid=64938 : : -: : : : : 29&cid=6494462&cid=6495189&stat_type=campdate&csrf_token=pRFezf5GIV : : -: : : : : r0wz8J : : -: : : : 77 : : : -: : : : : =Yes&types=days&cid=4537562 : : -: : : : : =Yes&types=days&cid=3353427 : : -: : : : 01 : : : -: : : : : 7 : : -: : : : : 2 : : -: : : : 52 : : : -: : : : : 5 : : -: : : : : =Yes&types=days&cid=6932408 : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=7162500 : : : -: : : : howCampStat&cid=1714344&offline_stat=0&online_stat=0&stat_periods= : owCampStat&cid=1714344&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : 2013-06-01:2013-06-30,2013-06-28:2013-06-29,2013-06-27:2013-06-28& : 13-06-01:2013-06-30,2013-06-28:2013-06-29,2013-06-27:2013-06-28&det : : -: : : : detail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&goa : : -: : : : &goals=0&target_all=yes&show_banners_stat=1 : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : : vancedForecast : : -: : : : : 3 : : -: : : : : 4 : : -: : : : 66 : : : -: : : : : 1 : : -: : : : howCampStat&cid=5360801&offline_stat=0&online_stat=1&stat_periods= : : : -: : : : &types=total&page=&group=day&y1=2012&m1=07&d1=02&y2=2013&m2=07&d2= : : : -: : : : 02 : : : -: : : : : 5 : : -: : : : t : mpleCamp : : -: : : : : 5 : : -: : : : : 2 : : -: : : : : 5 : : -: : : : : 6 : : -: : : : : 0 : : -: : : : : 3 : : -: : : : : 4 : : -: : : : : 9 : : -: : : : : =Yes&types=days&cid=7032884 : : -: : : : ts : : : -: : : : howCamp&cid=6411626&tab=off : owCamp&cid=6411626 : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : : 3 : : -: : : : : 6 : : -: : : : : 5 : : -: : : : : 8 : : -: : : : : =Yes&types=days&cid=6844274 : : -: : : : l=Yes&cid=2139196&ulogin=onclinic-sm&showCampStat=cmd&types=days&t : owCampStat&cid=2139196&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : arget_all=1&d1=01&m1=06&y1=13&d2=01&m2=07&y2=13&group=day : 13-06-01:2013-07-02,2013-06-21:2013-07-02,2013-06-24:2013-07-02&ulo : : -: : : : : gin=onclinic-sm&detail=yes&page=&y1=2013&m1=06&d1=01&y2=2013&m2=07& : : -: : : : : d2=02&group=day&goals=0&target_all=yes : : -: : : : : 6 : : -: : : : : 1 : : -: : : : 75 : : : -: : : : : 6 : : -: : : : 148542&stat_type=pages&target_all=1 : =Yes&cid=7148542&showCampStat=cmd&types=days&target_all=1 : : -: : : : : owCampStat&cid=4917291&detail=Yes&types=days : : -: : : : : 1 : : -: : : : 7-.31620-1.1372703844295129.198343986.198343986..53121502.1406.512 : 7 : : -: : : : l=Yes&types=days&cid=7041629 : : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6027971 : : -: : : : 20 : : : -: : : : : vancedForecast : : -: : : : : =Yes&types=days&cid=7034128 : : -: : : : =total&target_all=1&page=1&cmd=showCampStat&cid=3016175&show_banne : owCampStat&cid=3016175&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : rs_stat=1 : 12-02-03:2012-02-04,2011-08-01:2011-09-04,2010-12-29:2011-01-02&typ : : -: : : : : es=total&page=&group=day&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : 13411=1#list : : -: : : : l=Yes&types=days&cid=88689 : : : -: : : : : 88.0 : : -: : : : howCamps&mediaType=text&ulogin=super-svictor : ype=campdate&&ulogin=super-svictor : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=6940624 : : -: : : : : &mediaType=text&firstbanner=easy : : -: : : : : dirlevel=1372663385.0&cmd=showCamp&cid=2139196&ulogin=onclinic-sm : : -: : : : 97&ulogin=mirkli-webmart : : : -: : : : : 6 : : -: : : : : 0 : : -: : : : l=Yes&types=days&cid=4644721 : owCampStat&cid=4644721&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 13-05-27:2013-05-28,2011-09-12:2011-09-12,2011-06-05:2011-07-05&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&goa : : -: : : : : ls=0&target_all=yes&show_banners_stat=1 : : -: : : : : 7 : : -: : : : : 3 : : -: : : : : 5 : : -: : : : : dirlevel=1372412588.0&authredirlevel=1372408461.0&authredirlevel=13 : : -: : : : : 72403851.0&authredirlevel=1372339893.0&authredirlevel=1372335674.0& : : -: : : : : authredirlevel=1372331762.0&authredirlevel=1372323148.0&authredirle : : -: : : : : vel=1372313398.0&authredirlevel=1372309630.0&authredirlevel=1372252 : : -: : : : : 641.0&authredirlevel=1372248697.0&authredirlevel=1372244608.0&authr : : -: : : : : edirlevel=1372240548.0&authredirlevel=1372231517.0&authredirlevel=1 : : -: : : : : 372227248.0&authredirlevel=1372074352.0&authredirlevel=1371476832.0 : : -: : : : : &cmd=showCamp&tab=active&cid=6051926 : : -: : : : howCamp&cid=6501536 : : : -: : : : howCampStat&cid=7022493&detail=Yes&target_all=1&types=days&show_ba : : : -: : : : nners_stat=1&page=2 : : : -: : : : : &mediaType=text : : -: : : : : =Yes&types=days&cid=7041629 : : -: : : : : 9 : : -: : : : : 3 : : -: : : : l=Yes&types=days&cid=2704830 : : : -: : : : : =Yes&types=days&cid=6100946 : : -: : : : : 0 : : -: : : : : 8 : : -: : : : : 37.0&authredirlevel=1371612009.0&authredirlevel=1370955451.0&cmd=sh : : -: : : : : owCamp&cid=5477450 : : -: : : : 18 : : : -: : : : 17 : : : -: : : : : 0 : : -: : : : : 6 : : -: : : : : dirlevel=1374312836.0&cmd=showCamps : : -: : : : : 5 : : -: : : : 57 : : : -: : : : : 5 : : -: : : : : 3 : : -: : : : l=Yes&cid=2870560&showCampStat=cmd&types=days&target_all=1&d1=02&m : 0 : : -: : : : 1=07&y1=13&d2=02&m2=07&y2=13&group=day : : : -: : : : : 0 : : -: : : : : 3 : : -: : : : &ulogin= : ulogin= : : -: : : : 12 : : : -: : : : oInformers.html : : : -: : : : howCamp&cid=4693423&tab=active : owCamp&cid=4693423&tab=wait : : -: : : : : 2 : : -: : : : : 1 : : -: : : : : 9 : : -: : : : : 11.0&authredirlevel=1372755166.0&cmd=showClients : : -: : : : : 1 : : -: : : : : 1 : : -: : : : : 3 : : -: : : : : 066835&cmd=showCamp : : -: : : : : 0 : : -: : : : 6 : : : -: : : : : 3 : : -: : : : : 7 : : -: : : : howCamp&cid=970693 : owCamp&cid=2997085 : : -: : : : : 8 : : -: : : : 1&mediaType=text : : : -: : : : : 6 : : -: : : : : 3 : : -: : : : : 1 : : -: : : : 79 : owCamp&cid=7108679&tab=&page=2 : : -: : : : : =Yes&types=days&cid=7169384 : : -: : : : oInformers.html : : : -: : : : : dirlevel=1372772163.0&cmd=showCamps&tab=all&ulogin=spb-tester1 : : -: : : : : 8 : : -: : : : : 0 : : -: : : : 61 : : : -: : : : : vancedForecast : : -: : : : : =Yes&types=days&cid=6974488 : : -: : : : 84 : : : -: : : : : owCamp&cid=6833286 : : -: : : : : 5 : : -: : : : : 4 : : -: : : : howCamps&ulogin=major-auto-RW : ype=campdate&&ulogin=major-auto-RW : : -: : : : : 9 : : -: : : : ditBannerEasy&cid=6939766&bid=178815009 : dirlevel=1372759245.0&&cid=6939766&cmd=showCamp : : -: : : : ned : : : -: : : : ditCamp&cid=2870560 : owCamp&cid=2870560 : : -: : : : : 8 : : -: : : : howCampStat&cid=7058631&detail=Yes&types=days : 58631&stat_type=pages&target_all=1 : : -: : : : : =Yes&types=days&cid=3714884 : : -: : : : 97 : : : -: : : : : 0 : : -: : : : bids=5756034&cid=1840733 : : : -: : : : : =Yes&types=days&cid=4641928 : : -: : : : 91 : : : -: : : : : 9 : : -: : : : : 3 : : -: : : : : 49.0&authredirlevel=1372754333.0&authredirlevel=1372750347.0&cmd=sh : : -: : : : : owCamp&cid=5694785&ulogin=otzyvavto : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0#6136521 : : -: : : : l=Yes&types=days&cid=7153732 : 53732&phrasedate=Yes&target_all=1 : : -: : : : : 4 : : -: : : : : 1 : : -: : : : : 1 : : -: : : : : =213&text_geo=Москва : : -: : : : : 0 : : -: : : : : 6 : : -: : : : : ype=campdate : : -: : : : : 3 : : -: : : : : &mediaType=text&firstbanner=std : : -: : : : : 4 : : -: : : : : 8 : : -: : : : 92 : owCampStat&cid=6486492&detail=Yes&types=days : : -: : : : l=Yes&types=days&cid=6975816 : 6 : : -: : : : : owCamps : : -: : : : : 9 : : -: : : : : 6 : : -: : : : : 1 : : -: : : : 75 : : : -: : : : : =Yes&types=days&cid=6942376 : : -: : : : 68 : : : -: : : : : 7 : : -: : : : : 4 : : -: : : : : 9 : : -: : : : l=Yes&types=days&cid=6851682 : days&page=1&cmd=showCampStat&detail=Yes&target_all=1&cid=6851682&sh : : -: : : : : ow_banners_stat=1 : : -: : : : : =Yes&types=days&cid=2704830 : : -: : : : 21 : : : -: : : : 04 : owCampStat&cid=7057804&detail=Yes&types=days : : -: : : : : 2 : : -: : : : : =Yes&types=days&cid=6486629 : : -: : : : 73 : : : -: : : : : 7 : : -: : : : : 8 : : -: : : : 31 : owCamp&cid=6839731 : : -: : : : 066272&phrasedate=Yes&target_all=1 : : : -: : : : howCampStat&cid=2803428&offline_stat=0&online_stat=0&stat_periods= : 8 : : -: : : : 2013-06-01:2013-06-24,2013-06-01:2013-06-14,2013-06-03:2013-06-03& : : : -: : : : phrasedate=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group : : : -: : : : =day&goals=0&target_all=yes : : : -: : : : : 5 : : -: : : : : 86.0&authredirlevel=1372751903.0&cmd=showCamp&cid=3055906 : : -: : : : : 3 : : -: : : : : 0 : : -: : : : : =Yes&types=days&cid=6157316 : : -: : : : : =Yes&types=days&cid=3502476 : : -: : : : : addtime&reverse=1&ws_place=0&ws_time=1&ws_cid=0&ws_done=0#5809640 : : -: : : : : =Yes&types=days&cid=6860824 : : -: : : : edirlevel=1372746442.0&authredirlevel=1372714097.0&authredirlevel= : owCampStat&cid=3226358&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : 1372698041.0&authredirlevel=1372693736.0&cmd=showCampStat&cid=3226 : 13-06-24:2013-07-02,&phrasedate=yes&page=&y1=2013&m1=06&d1=24&y2=20 : : -: : : : 358&offline_stat=0&online_stat=0&stat_periods=&phrasedate=yes&page : 13&m2=07&d2=02&group=day&goals=0&target_all=yes : : -: : : : =&y1=2013&m1=06&d1=24&y2=2013&m2=07&d2=01&group=day&goals=0&target : : : -: : : : _all=yes : : : -: : : : : 5 : : -: : : : howCamps&ulogin=dc12a : owCampStat&cid=7041500&stat_type=campdate&ulogin=dc12a&csrf_token=Q : : -: : : : : un6g1847J4TOMJW : : -: : : : howCamp&tab=all&cid=6831216 : owCamp&cid=6831216&tab=active : : -: : : : : 3 : : -: : : : : 064001&cmd=showCamp : : -: : : : : =Yes&types=days&cid=7165103 : : -: : : : : 6 : : -: : : : : ype=campdate : : -: : : : 04 : : : -: : : : 622777;offline_stat=0;online_stat=1;stat_periods=2013-06-01:2013-0 : : : -: : : : 7-02,2013-04-01:2013-06-28,2013-03-25:2013-06-28;detail=yes;page=; : : : -: : : : y1=2013;m1=06;d1=01;y2=2013;m2=07;d2=02;group=day;goals=0;target_a : : : -: : : : ll=yes : : : -: : : : : =Yes&types=days&cid=7066904 : : -: : : : : 90.0&cmd=showCamp&cid=7142178 : : -: : : : : =Yes&types=days&cid=4917905 : : -: : : : 171931&phrasedate=Yes&target_all=1 : : : -: : : : : 9 : : -: : : : l=Yes&types=days&cid=5967378 : owCampStat&cid=5967378&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-06-18:2013-07-02,2013-06-18:2013-06-24,2013-06-24:2013-06-30&det : : -: : : : : ail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day&goa : : -: : : : : ls=0&target_all=yes : : -: : : : : =Yes&types=days&cid=5685110 : : -: : : : : 3 : : -: : : : : owCamp&cid=7163322#197090829 : : -: : : : : 1 : : -: : : : l=Yes&types=days&cid=7087520 : : : -: : : : : =Yes&types=days&cid=7064245 : : -: : : : 36 : : : -: : : : : 4 : : -: : : : : owCamp&cid=2217231 : : -: : : : : 064001&cmd=showCamp : : -: : : : : 0 : : -: : : : : 2 : : -: : : : : 4 : : -: : : : : 0 : : -: : : : : 9 : : -: : : : 68 : : : -: : : : howCamp&cid=2838772&tab=off : owCamp&cid=2838772&tab=active : : -: : : : : 8 : : -: : : : : 5 : : -: : : : l=Yes&types=days&cid=7036399 : : : -: : : : 73 : : : -: : : : howCamp&cid=6155736#165459179 : : : -: : : : : 5 : : -: : : : : 2 : : -: : : : &ulogin= : : : -: : : : : 1 : : -: : : : : 0 : : -: : : : : 8 : : -: : : : 61 : : : -: : : : l=Yes&types=days&cid=7051349 : : : -: : : : : =Yes&types=days&cid=7065579 : : -: : : : : 5 : : -: : : : 02 : : : -: : : : : 27.0&cmd=showCamp&cid=4670220 : : -: : : : : 9 : : -: : : : : 1 : : -: : : : : =Yes&types=days&cid=4764350 : : -: : : : : 6 : : -: : : : : 3 : : -: : : : 35 : owCampStat&cid=6921735&detail=Yes&types=days : : -: : : : : =Yes&types=days&cid=6851682 : : -: : : : : owCampStat&stat_type=by_agency_clients : : -: : : : : =Yes&types=days&cid=7055483 : : -: : : : 14 : : : -: : : : : 4 : : -: : : : 69 : : : -: : : : : 4 : : -: : : : : campdate : : -: : : : : 2 : : -: : : : : 9 : : -: : : : : 4 : : -: : : : : 3 : : -: : : : ditCamp&cid=7100989 : 00989;offline_stat=0;online_stat=1;stat_periods=2013-01-10:2013-01- : : -: : : : : 28,2013-01-10:2013-01-27,2013-01-10:2013-01-26;detail=yes;page=;y1= : : -: : : : : 2013;m1=06;d1=22;y2=2013;m2=07;d2=22;group=day;goals=0;target_all=y : : -: : : : : es : : -: : : : : 2 : : -: : : : : 4 : : -: : : : 35 : : : -: : : : : 7 : : -: : : : : g14AZEA.pl?authredirlevel=1372773289.0&cmd=showCamps&ulogin=eslucha : : -: : : : : k : : -: : : : : &mediaType=text&from=fp_bbb : : -: : : : howCamps : 7 : : -: : : : 92 : : : -: : : : : 02.0&cmd=showClients : : -: : : : : 1 : : -: : : : howCamp&cid=7090606 : owCampStat&cid=7090606&detail=Yes&types=days : : -: : : : 61 : : : -: : : : : =Yes&types=days&cid=6994484 : : -: : : : : 3 : : -: : : : : 4 : : -: : : : : 6 : : -: : : : : 4 : : -: : : : : =Yes&types=days&cid=6796978&ulogin=rfidhy : : -: : : : : 3 : : -: : : : : =Yes&types=days&cid=5196539 : : -: : : : : owCamps&ulogin=filterlife-skyboom : : -: : : : : 1 : : -: : : : : 3 : : -: : : : : =Yes&types=days&cid=7010416 : : -: : : : : owCamps : : -: : : : : owClients : : -: : : : 394451&phrasedate=Yes&target_all=1 : owCampStat&cid=4394451&offline_stat=0&online_stat=0&stat_periods=20 : : -: : : : : 12-01-01:2012-11-21,2012-01-01:2012-07-31,2012-03-23:2012-03-23&phr : : -: : : : : asedate=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m2=07&d2=02&group=day : : -: : : : : &goals=0&target_all=yes : : -: : : : howCampStat&cid=5792419&detail=Yes&types=days : owCampStat&cid=5792419&offline_stat=0&online_stat=1&stat_periods=20 : : -: : : : : 13-02-14:2013-02-14,&detail=yes&page=&y1=2013&m1=07&d1=02&y2=2013&m : : -: : : : : 2=07&d2=02&group=day&goals=0&target_all=yes : : -: : : : : 3 : : -: : : : : 053221&cmd=showCamp : : -: : : : : 6 : : -: : : : : owCamps : : -+------+------+------+--------------------------------------------------------------------+---------------------------------------------------------------------+-------+ -1000 tuples (4m 55s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND T raficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (308.781ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND T raficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (24.550ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND T raficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (132.448ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AN D NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (252.500ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AN D NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (24.141ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AN D NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (25.051ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -+----------------------------+-----------+ -+============================+===========+ -+----------------------------+-----------+ -652 tuples (3.6s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -+----------------------------+-----------+ -+============================+===========+ -+----------------------------+-----------+ -652 tuples (2.0s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -+----------------------------+-----------+ -+============================+===========+ -+----------------------------+-----------+ -652 tuples (1.9s) -sql>\q -stop time: Пн. сент. 16 16:31:17 MSK 2013 diff --git a/benchmark/monetdb/log/log_10m_corrected_1 b/benchmark/monetdb/log/log_10m_corrected_1 deleted file mode 100644 index e6ccc6a8ade..00000000000 --- a/benchmark/monetdb/log/log_10m_corrected_1 +++ /dev/null @@ -1,3199 +0,0 @@ -start time: Tue Sep 17 09:00:01 MSK 2013 -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT count(*) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m; -+----------+ -+==========+ -+----------+ -1 tuple (3.851ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT count(*) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m; -+----------+ -+==========+ -+----------+ -1 tuple (3.389ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT count(*) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m; -+----------+ -+==========+ -+----------+ -1 tuple (3.633ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID <> 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE AdvEngineID <> 0; -+--------+ -+========+ -+--------+ -1 tuple (400.058ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID <> 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE AdvEngineID <> 0; -+--------+ -+========+ -+--------+ -1 tuple (45.117ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT count(*) FROM hits_10m WHERE AdvEngineID <> 0; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE AdvEngineID <> 0; -+--------+ -+========+ -+--------+ -1 tuple (7.999ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+---------+----------+--------------------------+ -+=========+==========+==========================+ -+---------+----------+--------------------------+ -1 tuple (207.544ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+---------+----------+--------------------------+ -+=========+==========+==========================+ -+---------+----------+--------------------------+ -1 tuple (22.219ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -+---------+----------+--------------------------+ -+=========+==========+==========================+ -+---------+----------+--------------------------+ -1 tuple (33.007ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT sum(UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_10m; -overflow in calculation. -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT sum(UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_10m; -overflow in calculation. -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT sum(UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(UserID) FROM hits_10m; -overflow in calculation. -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (873.236ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (614.270ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT count(DISTINCT UserID) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT UserID) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (564.001ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (3.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (2.8s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -+---------+ -+=========+ -+---------+ -1 tuple (2.9s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_10m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (191.616ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_10m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (4.704ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT min(EventDate), max(EventDate) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT min(EventDate), max(EventDate) FROM hits_10m; -+------------+------------+ -+============+============+ -+------------+------------+ -1 tuple (8.579ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID <> 0 GROUP B 0 GROUP BY  AdvEngineID ORDER BY cou 0 GROUP BY AdvEngineID ORDER BY coun t(*) DESC; -+-------------+-------+ -+=============+=======+ -+-------------+-------+ -9 tuples (56.658ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID <> 0 GROUP B 0 GROUP BY  AdvEngineID ORDER BY cou 0 GROUP BY AdvEngineID ORDER BY coun t(*) DESC; -+-------------+-------+ -+=============+=======+ -+-------------+-------+ -9 tuples (49.444ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID <> 0 GROUP B 0 GROUP BY  AdvEngineID ORDER BY cou 0 GROUP BY AdvEngineID ORDER BY coun t(*) DESC; -+-------------+-------+ -+=============+=======+ -+-------------+-------+ -9 tuples (35.463ms) -sql>\q --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY Regio \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY Regio \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY Regio \q --- агрегация, среднее количество ключей.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), c \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), c \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), c \q --- агрегация, среднее количество ключей, несколько агрегатных функций.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE <, count(DISTINCT UserID) AS u FROM hits_10m WHERE M obilePhoneModel <> '' GRO '' GROU P BY MobilePhoneModel ORD '' GROUP BY MobilePhoneModel ORDE R BY u DESC LIMIT 10; -+------------------+-------+ -+==================+=======+ -+------------------+-------+ -10 tuples (458.164ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE <, count(DISTINCT UserID) AS u FROM hits_10m WHERE M obilePhoneModel <> '' GRO '' GROU P BY MobilePhoneModel ORD '' GROUP BY MobilePhoneModel ORDE R BY u DESC LIMIT 10; -+------------------+-------+ -+==================+=======+ -+------------------+-------+ -10 tuples (319.166ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE <, count(DISTINCT UserID) AS u FROM hits_10m WHERE M obilePhoneModel <> '' GRO '' GROU P BY MobilePhoneModel ORD '' GROUP BY MobilePhoneModel ORDE R BY u DESC LIMIT 10; -+------------------+-------+ -+==================+=======+ -+------------------+-------+ -10 tuples (413.914ms) -sql>\q --- мощная фильтрация по строкам, затем агрегация по строкам.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hi '' GROUP BY MobileP <_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePh one, MobilePhoneModel ORD '' GROUP BY MobilePhone, MobilePhoneModel ORDE R BY u DESC LIMIT 10; -+-------------+------------------+-------+ -+=============+==================+=======+ -+-------------+------------------+-------+ -10 tuples (344.021ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hi '' GROUP BY MobileP <_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePh one, MobilePhoneModel ORD '' GROUP BY MobilePhone, MobilePhoneModel ORDE R BY u DESC LIMIT 10; -+-------------+------------------+-------+ -+=============+==================+=======+ -+-------------+------------------+-------+ -10 tuples (314.183ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hi '' GROUP BY MobileP <_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePh one, MobilePhoneModel ORD '' GROUP BY MobilePhone, MobilePhoneModel ORDE R BY u DESC LIMIT 10; -+-------------+------------------+-------+ -+=============+==================+=======+ -+-------------+------------------+-------+ -10 tuples (348.057ms) -sql>\q --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROU '' GROUP  BY SearchPhrase ORDER BY < SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------+------+ -+=======================+======+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROU '' GROUP  BY SearchPhrase ORDER BY < SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------+------+ -+=======================+======+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROU '' GROUP  BY SearchPhrase ORDER BY < SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -+-----------------------+------+ -+=======================+======+ -write error -sql>\q --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE Sear '' GROUP BY Se '' GROUP BY Sea rchPhrase ORDER BY u DESC '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+-----------------------+------+ -+=======================+======+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE Sear '' GROUP BY Se '' GROUP BY Sea rchPhrase ORDER BY u DESC '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+-----------------------+------+ -+=======================+======+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE Sear '' GROUP BY Se '' GROUP BY Sea rchPhrase ORDER BY u DESC '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; -+-----------------------+------+ -+=======================+======+ -write error -sql>\q --- агрегация чуть сложнее.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchP '' GROUP BY Searc '' GROUP BY Search EngineID, SearchPhrase OR '' GROUP BY SearchEngineID, SearchPhrase ORD ER BY count(*) DESC LIMIT \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchP '' GROUP BY Searc '' GROUP BY Search EngineID, SearchPhrase OR '' GROUP BY SearchEngineID, SearchPhrase ORD ER BY count(*) DESC LIMIT \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchP '' GROUP BY Searc '' GROUP BY Search EngineID, SearchPhrase OR '' GROUP BY SearchEngineID, SearchPhrase ORD ER BY count(*) DESC LIMIT \q --- агрегация по числу и строке, большое количество ключей.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) D < FROM hits_10m GROUP BY UserID ORDER BY count(*) DE SC LIMIT 10; -+---------------------+------+ -+=====================+======+ -+---------------------+------+ -10 tuples (886.465ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) D < FROM hits_10m GROUP BY UserID ORDER BY count(*) DE SC LIMIT 10; -+---------------------+------+ -+=====================+======+ -+---------------------+------+ -10 tuples (711.899ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) D < FROM hits_10m GROUP BY UserID ORDER BY count(*) DE SC LIMIT 10; -+---------------------+------+ -+=====================+======+ -+---------------------+------+ -10 tuples (732.900ms) -sql>\q --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, Sear \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, Sear \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, Sear \q --- ещё более сложная агрегация.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, Sear \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, Sear \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, Sear \q --- то же самое, но без сортировки.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count( <(minute from EventTime) AS m, SearchPhrase, count(* ) FROM hits_10m GROUP BY < m, SearchPhrase, count(*) FROM hits_10m GROUP BY U serID, m, SearchPhrase OR < FROM hits_10m GROUP BY UserID, m, SearchPhrase ORD ER BY count(*) DESC LIMIT \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count( <(minute from EventTime) AS m, SearchPhrase, count(* ) FROM hits_10m GROUP BY < m, SearchPhrase, count(*) FROM hits_10m GROUP BY U serID, m, SearchPhrase OR < FROM hits_10m GROUP BY UserID, m, SearchPhrase ORD ER BY count(*) DESC LIMIT \q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count( <(minute from EventTime) AS m, SearchPhrase, count(* ) FROM hits_10m GROUP BY < m, SearchPhrase, count(*) FROM hits_10m GROUP BY U serID, m, SearchPhrase OR < FROM hits_10m GROUP BY UserID, m, SearchPhrase ORD ER BY count(*) DESC LIMIT \q --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -+--------+ -+========+ -+--------+ -0 tuples (331.309ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -+--------+ -+========+ -+--------+ -0 tuples (0.618ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT UserID FROM hits_10m WHERE UserID = 1234567890; -+--------+ -+========+ -+--------+ -0 tuples (85.817ms) -sql>\q --- мощная фильтрация по столбцу типа UInt64.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+------+ -+======+ -+------+ -1 tuple (2.0s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+------+ -+======+ -+------+ -1 tuple (56.549ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; -+------+ -+======+ -+------+ -1 tuple (92.920ms) -sql>\q --- фильтрация по поиску подстроки в строке.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%me  '' GROUP BY SearchPhrase '' GROUP BY SearchPhrase ORDER BY count(*) DESC LI <'' GROUP BY SearchPhrase ORDER BY count(*) DESC LIM IT 10; -+------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+------+ -+========================================================================+=========================================================================================+======+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%me  '' GROUP BY SearchPhrase '' GROUP BY SearchPhrase ORDER BY count(*) DESC LI <'' GROUP BY SearchPhrase ORDER BY count(*) DESC LIM IT 10; -+------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+------+ -+========================================================================+=========================================================================================+======+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%me  '' GROUP BY SearchPhrase '' GROUP BY SearchPhrase ORDER BY count(*) DESC LI <'' GROUP BY SearchPhrase ORDER BY count(*) DESC LIM IT 10; -+------------------------------------------------------------------------+-----------------------------------------------------------------------------------------+------+ -+========================================================================+=========================================================================================+======+ -write error -sql>\q --- вынимаем большие столбцы, фильтрация по строке.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL <> '%.yandex.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT '%.yandex.%' AND '%.yandex.%' AND S earchPhrase <> '' GROUP B <URL <> '%.yandex.%' AND SearchPhrase <> '' GROUP BY  SearchPhrase ORDER BY co <archPhrase <> '' GROUP BY SearchPhrase ORDER BY cou nt(*) DESC LIMIT 10; -+----------------------------+-------------------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+------+------+ -+============================+=============================================================+=============================================================================================================================================================================================================================================================================================================================================================+======+======+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL <> '%.yandex.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT <X(URL), MAX(Title), count(*) AS c, count(DISTINCT U serID) FROM hits_10m WHER <*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE  Title LIKE '%??????%' AN <erID) FROM hits_10m WHERE Title LIKE '%??????%' AND  URL <> '%.yandex.%' AND <Title LIKE '%??????%' AND URL <> '%.yandex.%' AND S earchPhrase <> '' GROUP B <URL <> '%.yandex.%' AND SearchPhrase <> '' GROUP BY  SearchPhrase ORDER BY co <archPhrase <> '' GROUP BY SearchPhrase ORDER BY cou nt(*) DESC LIMIT 10; -+----------------------------+-------------------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+------+------+ -+============================+=============================================================+=============================================================================================================================================================================================================================================================================================================================================================+======+======+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL <> '%.yandex.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT <X(URL), MAX(Title), count(*) AS c, count(DISTINCT U serID) FROM hits_10m WHER <*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE  Title LIKE '%??????%' AN <erID) FROM hits_10m WHERE Title LIKE '%??????%' AND  URL <> '%.yandex.%' AND <Title LIKE '%??????%' AND URL <> '%.yandex.%' AND S earchPhrase <> '' GROUP B <URL <> '%.yandex.%' AND SearchPhrase <> '' GROUP BY  SearchPhrase ORDER BY co <archPhrase <> '' GROUP BY SearchPhrase ORDER BY cou nt(*) DESC LIMIT 10; -+----------------------------+-------------------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+------+------+ -+============================+=============================================================+=============================================================================================================================================================================================================================================================================================================================================================+======+======+ -write error -sql>\q --- чуть больше столбцы.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMI <WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT  10; -+---------------------+------------+--------------+-----------+----------------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+----------------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+----------------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+=====================+============+==============+===========+============================+============+===========+============+==========+=====================+==============+======+===========+=======================================================================================+=================================================================================================================+=========+===================+=================+===============+=============+=================+==================+=================+============+============+=============+==========+==========+================+================+==============+==================+==========+=============+==================+========+=============+================+================+==============+=============+=============+===================+====================+================+============================+=====================+=====================+=====================+=====================+=============+=============+========+============+=============+=====================+=============+============+==============+=========+=============+===============+==========+==========+============================+======+======+========+===========+===========+============+============+============+===============+=================+================+===============+==============+===========+============+===========+===============+=====================+===================+=============+=======================+==================+============+==============+===============+=================+=====================+====================+==============+==================+===========+===========+=============+============+=========+=========+==========+======================+=====================+======+============+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMI <WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT  10; -+---------------------+------------+--------------+-----------+----------------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+----------------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+----------------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+=====================+============+==============+===========+============================+============+===========+============+==========+=====================+==============+======+===========+=======================================================================================+=================================================================================================================+=========+===================+=================+===============+=============+=================+==================+=================+============+============+=============+==========+==========+================+================+==============+==================+==========+=============+==================+========+=============+================+================+==============+=============+=============+===================+====================+================+============================+=====================+=====================+=====================+=====================+=============+=============+========+============+=============+=====================+=============+============+==============+=========+=============+===============+==========+==========+============================+======+======+========+===========+===========+============+============+============+===============+=================+================+===============+==============+===========+============+===========+===============+=====================+===================+=============+=======================+==================+============+==============+===============+=================+=====================+====================+==============+==================+===========+===========+=============+============+=========+=========+==========+======================+=====================+======+============+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMI <WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT  10; -+---------------------+------------+--------------+-----------+----------------------------+------------+-----------+------------+----------+---------------------+--------------+------+-----------+---------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------+---------+-------------------+-----------------+---------------+-------------+-----------------+------------------+-----------------+------------+------------+-------------+----------+----------+----------------+----------------+--------------+------------------+----------+-------------+------------------+--------+-------------+----------------+----------------+--------------+-------------+-------------+-------------------+--------------------+----------------+----------------------------+---------------------+---------------------+---------------------+---------------------+-------------+-------------+--------+------------+-------------+---------------------+-------------+------------+--------------+---------+-------------+---------------+----------+----------+----------------------------+------+------+--------+-----------+-----------+------------+------------+------------+---------------+-----------------+----------------+---------------+--------------+-----------+------------+-----------+---------------+---------------------+-------------------+-------------+-----------------------+------------------+------------+--------------+---------------+-----------------+---------------------+--------------------+--------------+------------------+-----------+-----------+-------------+------------+---------+---------+----------+----------------------+---------------------+------+------------+ -+=====================+============+==============+===========+============================+============+===========+============+==========+=====================+==============+======+===========+=======================================================================================+=================================================================================================================+=========+===================+=================+===============+=============+=================+==================+=================+============+============+=============+==========+==========+================+================+==============+==================+==========+=============+==================+========+=============+================+================+==============+=============+=============+===================+====================+================+============================+=====================+=====================+=====================+=====================+=============+=============+========+============+=============+=====================+=============+============+==============+=========+=============+===============+==========+==========+============================+======+======+========+===========+===========+============+============+============+===============+=================+================+===============+==============+===========+============+===========+===============+=====================+===================+=============+=======================+==================+============+==============+===============+=================+=====================+====================+==============+==================+===========+===========+=============+============+=========+=========+==========+======================+=====================+======+============+ -write error -sql>\q --- плохой запрос - вынимаем все столбцы.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY Event <M hits_10m WHERE SearchPhrase <> '' ORDER BY EventT ime LIMIT 10; -+------------------------------------+ -+====================================+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY Event <M hits_10m WHERE SearchPhrase <> '' ORDER BY EventT ime LIMIT 10; -+------------------------------------+ -+====================================+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY Event <M hits_10m WHERE SearchPhrase <> '' ORDER BY EventT ime LIMIT 10; -+------------------------------------+ -+====================================+ -write error -sql>\q --- большая сортировка.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY Searc <M hits_10m WHERE SearchPhrase <> '' ORDER BY Search Phrase LIMIT 10; -+------------------------------------------------------------------------------+ -+==============================================================================+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY Searc <M hits_10m WHERE SearchPhrase <> '' ORDER BY Search Phrase LIMIT 10; -+------------------------------------------------------------------------------+ -+==============================================================================+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY Searc <M hits_10m WHERE SearchPhrase <> '' ORDER BY Search Phrase LIMIT 10; -+------------------------------------------------------------------------------+ -+==============================================================================+ -write error -sql>\q --- большая сортировка по строкам.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY Event <M hits_10m WHERE SearchPhrase <> '' ORDER BY EventT ime, SearchPhrase LIMIT 1 <ase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10 ; -+------------------------------------+ -+====================================+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY Event <M hits_10m WHERE SearchPhrase <> '' ORDER BY EventT ime, SearchPhrase LIMIT 1 <ase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10 ; -+------------------------------------+ -+====================================+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY Event <M hits_10m WHERE SearchPhrase <> '' ORDER BY EventT ime, SearchPhrase LIMIT 1 <ase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10 ; -+------------------------------------+ -+====================================+ -write error -sql>\q --- большая сортировка по кортежу.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL <> '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL <ength(URL)) AS l, count(*) FROM hits_10m WHERE URL <> '' GROUP BY CounterID < FROM hits_10m WHERE URL <> '' GROUP BY CounterID H AVING count(*) > 100000 O <> '' GROUP BY CounterID HAVING count(*) > 100000 OR DER BY l DESC LIMIT 25; -+-----------+--------------------------+---------+ -+===========+==========================+=========+ -+-----------+--------------------------+---------+ -19 tuples (4.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL <> '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL <ength(URL)) AS l, count(*) FROM hits_10m WHERE URL <> '' GROUP BY CounterID < FROM hits_10m WHERE URL <> '' GROUP BY CounterID H AVING count(*) > 100000 O <> '' GROUP BY CounterID HAVING count(*) > 100000 OR DER BY l DESC LIMIT 25; -+-----------+--------------------------+---------+ -+===========+==========================+=========+ -+-----------+--------------------------+---------+ -19 tuples (2.7s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL <> '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL <ength(URL)) AS l, count(*) FROM hits_10m WHERE URL <> '' GROUP BY CounterID < FROM hits_10m WHERE URL <> '' GROUP BY CounterID H AVING count(*) > 100000 O <> '' GROUP BY CounterID HAVING count(*) > 100000 OR DER BY l DESC LIMIT 25; -+-----------+--------------------------+---------+ -+===========+==========================+=========+ -+-----------+--------------------------+---------+ -19 tuples (2.7s) -sql>\q --- считаем средние длины URL для крупных счётчиков.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, i <RING(Referer, POSITION('//' IN Referer) + 2), 1, if thenelse( (0 > POSITION(' <' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/ ' IN SUBSTRING(Referer, P <henelse( (0 > POSITION('/' IN SUBSTRING(Referer, PO SITION('//' IN Referer) + < IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITI <ITION('//' IN Referer) + 2)) - 1), 0, POSITIO N('/' IN SUBSTRING(Refere <)) - 1), 0, POSITION('/' IN SUBSTRING(Referer , POSITION('//' IN Refere <('/' IN SUBSTRING(Referer, POSITION('//' IN Referer ) + 2)) - 1 ) ) -more>\q -more>monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, i <RING(Referer, POSITION('//' IN Referer) + 2), 1, if thenelse( (0 > POSITION(' <' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/ ' IN SUBSTRING(Referer, P <henelse( (0 > POSITION('/' IN SUBSTRING(Referer, PO SITION('//' IN Referer) + < IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITI <ITION('//' IN Referer) + 2)) - 1), 0, POSITIO N('/' IN SUBSTRING(Refere <)) - 1), 0, POSITION('/' IN SUBSTRING(Referer , POSITION('//' IN Refere <('/' IN SUBSTRING(Referer, POSITION('//' IN Referer ) + 2)) - 1 ) ) -more>\q -more>monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, i <RING(Referer, POSITION('//' IN Referer) + 2), 1, if thenelse( (0 > POSITION(' <' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/ ' IN SUBSTRING(Referer, P <henelse( (0 > POSITION('/' IN SUBSTRING(Referer, PO SITION('//' IN Referer) + < IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITI <ITION('//' IN Referer) + 2)) - 1), 0, POSITIO N('/' IN SUBSTRING(Refere <)) - 1), 0, POSITION('/' IN SUBSTRING(Referer , POSITION('//' IN Refere <('/' IN SUBSTRING(Referer, POSITION('//' IN Referer ) + 2)) - 1 ) ) -more>\q -more>monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10 <r)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m  WHERE Referer <> '' GROU <AX(Referer) FROM hits_10m WHERE Referer <> '' GROUP  BY k HAVING count(*) > 1 <WHERE Referer <> '' GROUP BY k HAVING count(*) > 10 0000 ORDER BY l DESC LIMI <BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT  25; -syntax error, unexpected AS in: "as" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10 <r)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m  WHERE Referer <> '' GROU <AX(Referer) FROM hits_10m WHERE Referer <> '' GROUP  BY k HAVING count(*) > 1 <WHERE Referer <> '' GROUP BY k HAVING count(*) > 10 0000 ORDER BY l DESC LIMI <BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT  25; -syntax error, unexpected AS in: "as" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10 <r)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m  WHERE Referer <> '' GROU <AX(Referer) FROM hits_10m WHERE Referer <> '' GROUP  BY k HAVING count(*) > 1 <WHERE Referer <> '' GROUP BY k HAVING count(*) > 10 0000 ORDER BY l DESC LIMI <BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT  25; -syntax error, unexpected AS in: "as" -sql>\q --- то же самое, но с разбивкой по доменам.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidt <dth), sum(ResolutionWidth + 1), sum(ResolutionWidth  + 2), sum(ResolutionWidt <+ 1), sum(ResolutionWidth + 2), sum(ResolutionWidth  + 3), sum(ResolutionWidt <+ 2), sum(ResolutionWidth + 3), sum(ResolutionWidth  + 4), sum(ResolutionWidt <+ 3), sum(ResolutionWidth + 4), sum(ResolutionWidth  + 5), sum(ResolutionWidt <+ 4), sum(ResolutionWidth + 5), sum(ResolutionWidth  + 6), sum(ResolutionWidt <+ 5), sum(ResolutionWidth + 6), sum(ResolutionWidth  + 7), sum(ResolutionWidt <+ 6), sum(ResolutionWidth + 7), sum(ResolutionWidth  + 8), sum(ResolutionWidt <+ 7), sum(ResolutionWidth + 8), sum(ResolutionWidth  + 9), sum(ResolutionWidt <+ 8), sum(ResolutionWidth + 9), sum(ResolutionWidth  + 10), sum(ResolutionWid <+ 9), sum(ResolutionWidth + 10), sum(ResolutionWidt h + 11), sum(ResolutionWi <+ 10), sum(ResolutionWidth + 11), sum(ResolutionWid th + 12), sum(ResolutionW < + 11), sum(ResolutionWidth + 12), sum(ResolutionWi dth + 13), sum(Resolution <h + 12), sum(ResolutionWidth + 13), sum(ResolutionW idth + 14), sum(Resolutio <th + 13), sum(ResolutionWidth + 14), sum(Resolution Width + 15), sum(Resoluti <dth + 14), sum(ResolutionWidth + 15), sum(Resolutio nWidth + 16), sum(Resolut <idth + 15), sum(ResolutionWidth + 16), sum(Resoluti onWidth + 17), sum(Resolu <Width + 16), sum(ResolutionWidth + 17), sum(Resolut ionWidth + 18), sum(Resol <nWidth + 17), sum(ResolutionWidth + 18), sum(Resolu tionWidth + 19), sum(Reso <onWidth + 18), sum(ResolutionWidth + 19), sum(Resol utionWidth + 20), sum(Res <ionWidth + 19), sum(ResolutionWidth + 20), sum(Reso lutionWidth + 21), sum(Re <tionWidth + 20), sum(ResolutionWidth + 21), sum(Res olutionWidth + 22), sum(R <utionWidth + 21), sum(ResolutionWidth + 22), sum(Re solutionWidth + 23), sum( <lutionWidth + 22), sum(ResolutionWidth + 23), sum(R esolutionWidth + 24), sum <olutionWidth + 23), sum(ResolutionWidth + 24), sum( ResolutionWidth + 25), su <solutionWidth + 24), sum(ResolutionWidth + 25), sum (ResolutionWidth + 26), s <esolutionWidth + 25), sum(ResolutionWidth + 26), su m(ResolutionWidth + 27), <ResolutionWidth + 26), sum(ResolutionWidth + 27), s um(ResolutionWidth + 28), <(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29) <m(ResolutionWidth + 28), sum(ResolutionWidth + 29),  sum(ResolutionWidth + 30 <um(ResolutionWidth + 29), sum(ResolutionWidth + 30) , sum(ResolutionWidth + 3 <sum(ResolutionWidth + 30), sum(ResolutionWidth + 31 ), sum(ResolutionWidth + < sum(ResolutionWidth + 31), sum(ResolutionWidth + 3 2), sum(ResolutionWidth + <, sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth <), sum(ResolutionWidth + 33), sum(ResolutionWidth +  34), sum(ResolutionWidth <3), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidt <34), sum(ResolutionWidth + 35), sum(ResolutionWidth  + 36), sum(ResolutionWid < 35), sum(ResolutionWidth + 36), sum(ResolutionWidt h + 37), sum(ResolutionWi <+ 36), sum(ResolutionWidth + 37), sum(ResolutionWid th + 38), sum(ResolutionW < + 37), sum(ResolutionWidth + 38), sum(ResolutionWi dth + 39), sum(Resolution <h + 38), sum(ResolutionWidth + 39), sum(ResolutionW idth + 40), sum(Resolutio <th + 39), sum(ResolutionWidth + 40), sum(Resolution Width + 41), sum(Resoluti <dth + 40), sum(ResolutionWidth + 41), sum(Resolutio nWidth + 42), sum(Resolut <idth + 41), sum(ResolutionWidth + 42), sum(Resoluti onWidth + 43), sum(Resolu <Width + 42), sum(ResolutionWidth + 43), sum(Resolut ionWidth + 44), sum(Resol <nWidth + 43), sum(ResolutionWidth + 44), sum(Resolu tionWidth + 45), sum(Reso <onWidth + 44), sum(ResolutionWidth + 45), sum(Resol utionWidth + 46), sum(Res <ionWidth + 45), sum(ResolutionWidth + 46), sum(Reso lutionWidth + 47), sum(Re <tionWidth + 46), sum(ResolutionWidth + 47), sum(Res olutionWidth + 48), sum(R <utionWidth + 47), sum(ResolutionWidth + 48), sum(Re solutionWidth + 49), sum( <lutionWidth + 48), sum(ResolutionWidth + 49), sum(R esolutionWidth + 50), sum <olutionWidth + 49), sum(ResolutionWidth + 50), sum( ResolutionWidth + 51), su <solutionWidth + 50), sum(ResolutionWidth + 51), sum (ResolutionWidth + 52), s <esolutionWidth + 51), sum(ResolutionWidth + 52), su m(ResolutionWidth + 53), <ResolutionWidth + 52), sum(ResolutionWidth + 53), s um(ResolutionWidth + 54), <(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55) <m(ResolutionWidth + 54), sum(ResolutionWidth + 55),  sum(ResolutionWidth + 56 <um(ResolutionWidth + 55), sum(ResolutionWidth + 56) , sum(ResolutionWidth + 5 <sum(ResolutionWidth + 56), sum(ResolutionWidth + 57 ), sum(ResolutionWidth + < sum(ResolutionWidth + 57), sum(ResolutionWidth + 5 8), sum(ResolutionWidth + <, sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth <), sum(ResolutionWidth + 59), sum(ResolutionWidth +  60), sum(ResolutionWidth <9), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidt <60), sum(ResolutionWidth + 61), sum(ResolutionWidth  + 62), sum(ResolutionWid < 61), sum(ResolutionWidth + 62), sum(ResolutionWidt h + 63), sum(ResolutionWi <+ 62), sum(ResolutionWidth + 63), sum(ResolutionWid th + 64), sum(ResolutionW < + 63), sum(ResolutionWidth + 64), sum(ResolutionWi dth + 65), sum(Resolution <h + 64), sum(ResolutionWidth + 65), sum(ResolutionW idth + 66), sum(Resolutio <th + 65), sum(ResolutionWidth + 66), sum(Resolution Width + 67), sum(Resoluti <dth + 66), sum(ResolutionWidth + 67), sum(Resolutio nWidth + 68), sum(Resolut <idth + 67), sum(ResolutionWidth + 68), sum(Resoluti onWidth + 69), sum(Resolu <Width + 68), sum(ResolutionWidth + 69), sum(Resolut ionWidth + 70), sum(Resol <nWidth + 69), sum(ResolutionWidth + 70), sum(Resolu tionWidth + 71), sum(Reso <onWidth + 70), sum(ResolutionWidth + 71), sum(Resol utionWidth + 72), sum(Res <ionWidth + 71), sum(ResolutionWidth + 72), sum(Reso lutionWidth + 73), sum(Re <tionWidth + 72), sum(ResolutionWidth + 73), sum(Res olutionWidth + 74), sum(R <utionWidth + 73), sum(ResolutionWidth + 74), sum(Re solutionWidth + 75), sum( <lutionWidth + 74), sum(ResolutionWidth + 75), sum(R esolutionWidth + 76), sum <olutionWidth + 75), sum(ResolutionWidth + 76), sum( ResolutionWidth + 77), su <solutionWidth + 76), sum(ResolutionWidth + 77), sum (ResolutionWidth + 78), s <esolutionWidth + 77), sum(ResolutionWidth + 78), su m(ResolutionWidth + 79), <ResolutionWidth + 78), sum(ResolutionWidth + 79), s um(ResolutionWidth + 80), <(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81) <m(ResolutionWidth + 80), sum(ResolutionWidth + 81),  sum(ResolutionWidth + 82 <um(ResolutionWidth + 81), sum(ResolutionWidth + 82) , sum(ResolutionWidth + 8 <sum(ResolutionWidth + 82), sum(ResolutionWidth + 83 ), sum(ResolutionWidth + < sum(ResolutionWidth + 83), sum(ResolutionWidth + 8 4), sum(ResolutionWidth + <, sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth <), sum(ResolutionWidth + 85), sum(ResolutionWidth +  86), sum(ResolutionWidth <5), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidt <86), sum(ResolutionWidth + 87), sum(ResolutionWidth  + 88), sum(ResolutionWid < 87), sum(ResolutionWidth + 88), sum(ResolutionWidt h + 89) FROM hits_10m; -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+ -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -1 tuple (1.1s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidt <dth), sum(ResolutionWidth + 1), sum(ResolutionWidth  + 2), sum(ResolutionWidt <+ 1), sum(ResolutionWidth + 2), sum(ResolutionWidth  + 3), sum(ResolutionWidt <+ 2), sum(ResolutionWidth + 3), sum(ResolutionWidth  + 4), sum(ResolutionWidt <+ 3), sum(ResolutionWidth + 4), sum(ResolutionWidth  + 5), sum(ResolutionWidt <+ 4), sum(ResolutionWidth + 5), sum(ResolutionWidth  + 6), sum(ResolutionWidt <+ 5), sum(ResolutionWidth + 6), sum(ResolutionWidth  + 7), sum(ResolutionWidt <+ 6), sum(ResolutionWidth + 7), sum(ResolutionWidth  + 8), sum(ResolutionWidt <+ 7), sum(ResolutionWidth + 8), sum(ResolutionWidth  + 9), sum(ResolutionWidt <+ 8), sum(ResolutionWidth + 9), sum(ResolutionWidth  + 10), sum(ResolutionWid <+ 9), sum(ResolutionWidth + 10), sum(ResolutionWidt h + 11), sum(ResolutionWi <+ 10), sum(ResolutionWidth + 11), sum(ResolutionWid th + 12), sum(ResolutionW < + 11), sum(ResolutionWidth + 12), sum(ResolutionWi dth + 13), sum(Resolution <h + 12), sum(ResolutionWidth + 13), sum(ResolutionW idth + 14), sum(Resolutio <th + 13), sum(ResolutionWidth + 14), sum(Resolution Width + 15), sum(Resoluti <dth + 14), sum(ResolutionWidth + 15), sum(Resolutio nWidth + 16), sum(Resolut <idth + 15), sum(ResolutionWidth + 16), sum(Resoluti onWidth + 17), sum(Resolu <Width + 16), sum(ResolutionWidth + 17), sum(Resolut ionWidth + 18), sum(Resol <nWidth + 17), sum(ResolutionWidth + 18), sum(Resolu tionWidth + 19), sum(Reso <onWidth + 18), sum(ResolutionWidth + 19), sum(Resol utionWidth + 20), sum(Res <ionWidth + 19), sum(ResolutionWidth + 20), sum(Reso lutionWidth + 21), sum(Re <tionWidth + 20), sum(ResolutionWidth + 21), sum(Res olutionWidth + 22), sum(R <utionWidth + 21), sum(ResolutionWidth + 22), sum(Re solutionWidth + 23), sum( <lutionWidth + 22), sum(ResolutionWidth + 23), sum(R esolutionWidth + 24), sum <olutionWidth + 23), sum(ResolutionWidth + 24), sum( ResolutionWidth + 25), su <solutionWidth + 24), sum(ResolutionWidth + 25), sum (ResolutionWidth + 26), s <esolutionWidth + 25), sum(ResolutionWidth + 26), su m(ResolutionWidth + 27), <ResolutionWidth + 26), sum(ResolutionWidth + 27), s um(ResolutionWidth + 28), <(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29) <m(ResolutionWidth + 28), sum(ResolutionWidth + 29),  sum(ResolutionWidth + 30 <um(ResolutionWidth + 29), sum(ResolutionWidth + 30) , sum(ResolutionWidth + 3 <sum(ResolutionWidth + 30), sum(ResolutionWidth + 31 ), sum(ResolutionWidth + < sum(ResolutionWidth + 31), sum(ResolutionWidth + 3 2), sum(ResolutionWidth + <, sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth <), sum(ResolutionWidth + 33), sum(ResolutionWidth +  34), sum(ResolutionWidth <3), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidt <34), sum(ResolutionWidth + 35), sum(ResolutionWidth  + 36), sum(ResolutionWid < 35), sum(ResolutionWidth + 36), sum(ResolutionWidt h + 37), sum(ResolutionWi <+ 36), sum(ResolutionWidth + 37), sum(ResolutionWid th + 38), sum(ResolutionW < + 37), sum(ResolutionWidth + 38), sum(ResolutionWi dth + 39), sum(Resolution <h + 38), sum(ResolutionWidth + 39), sum(ResolutionW idth + 40), sum(Resolutio <th + 39), sum(ResolutionWidth + 40), sum(Resolution Width + 41), sum(Resoluti <dth + 40), sum(ResolutionWidth + 41), sum(Resolutio nWidth + 42), sum(Resolut <idth + 41), sum(ResolutionWidth + 42), sum(Resoluti onWidth + 43), sum(Resolu <Width + 42), sum(ResolutionWidth + 43), sum(Resolut ionWidth + 44), sum(Resol <nWidth + 43), sum(ResolutionWidth + 44), sum(Resolu tionWidth + 45), sum(Reso <onWidth + 44), sum(ResolutionWidth + 45), sum(Resol utionWidth + 46), sum(Res <ionWidth + 45), sum(ResolutionWidth + 46), sum(Reso lutionWidth + 47), sum(Re <tionWidth + 46), sum(ResolutionWidth + 47), sum(Res olutionWidth + 48), sum(R <utionWidth + 47), sum(ResolutionWidth + 48), sum(Re solutionWidth + 49), sum( <lutionWidth + 48), sum(ResolutionWidth + 49), sum(R esolutionWidth + 50), sum <olutionWidth + 49), sum(ResolutionWidth + 50), sum( ResolutionWidth + 51), su <solutionWidth + 50), sum(ResolutionWidth + 51), sum (ResolutionWidth + 52), s <esolutionWidth + 51), sum(ResolutionWidth + 52), su m(ResolutionWidth + 53), <ResolutionWidth + 52), sum(ResolutionWidth + 53), s um(ResolutionWidth + 54), <(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55) <m(ResolutionWidth + 54), sum(ResolutionWidth + 55),  sum(ResolutionWidth + 56 <um(ResolutionWidth + 55), sum(ResolutionWidth + 56) , sum(ResolutionWidth + 5 <sum(ResolutionWidth + 56), sum(ResolutionWidth + 57 ), sum(ResolutionWidth + < sum(ResolutionWidth + 57), sum(ResolutionWidth + 5 8), sum(ResolutionWidth + <, sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth <), sum(ResolutionWidth + 59), sum(ResolutionWidth +  60), sum(ResolutionWidth <9), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidt <60), sum(ResolutionWidth + 61), sum(ResolutionWidth  + 62), sum(ResolutionWid < 61), sum(ResolutionWidth + 62), sum(ResolutionWidt h + 63), sum(ResolutionWi <+ 62), sum(ResolutionWidth + 63), sum(ResolutionWid th + 64), sum(ResolutionW < + 63), sum(ResolutionWidth + 64), sum(ResolutionWi dth + 65), sum(Resolution <h + 64), sum(ResolutionWidth + 65), sum(ResolutionW idth + 66), sum(Resolutio <th + 65), sum(ResolutionWidth + 66), sum(Resolution Width + 67), sum(Resoluti <dth + 66), sum(ResolutionWidth + 67), sum(Resolutio nWidth + 68), sum(Resolut <idth + 67), sum(ResolutionWidth + 68), sum(Resoluti onWidth + 69), sum(Resolu <Width + 68), sum(ResolutionWidth + 69), sum(Resolut ionWidth + 70), sum(Resol <nWidth + 69), sum(ResolutionWidth + 70), sum(Resolu tionWidth + 71), sum(Reso <onWidth + 70), sum(ResolutionWidth + 71), sum(Resol utionWidth + 72), sum(Res <ionWidth + 71), sum(ResolutionWidth + 72), sum(Reso lutionWidth + 73), sum(Re <tionWidth + 72), sum(ResolutionWidth + 73), sum(Res olutionWidth + 74), sum(R <utionWidth + 73), sum(ResolutionWidth + 74), sum(Re solutionWidth + 75), sum( <lutionWidth + 74), sum(ResolutionWidth + 75), sum(R esolutionWidth + 76), sum <olutionWidth + 75), sum(ResolutionWidth + 76), sum( ResolutionWidth + 77), su <solutionWidth + 76), sum(ResolutionWidth + 77), sum (ResolutionWidth + 78), s <esolutionWidth + 77), sum(ResolutionWidth + 78), su m(ResolutionWidth + 79), <ResolutionWidth + 78), sum(ResolutionWidth + 79), s um(ResolutionWidth + 80), <(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81) <m(ResolutionWidth + 80), sum(ResolutionWidth + 81),  sum(ResolutionWidth + 82 <um(ResolutionWidth + 81), sum(ResolutionWidth + 82) , sum(ResolutionWidth + 8 <sum(ResolutionWidth + 82), sum(ResolutionWidth + 83 ), sum(ResolutionWidth + < sum(ResolutionWidth + 83), sum(ResolutionWidth + 8 4), sum(ResolutionWidth + <, sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth <), sum(ResolutionWidth + 85), sum(ResolutionWidth +  86), sum(ResolutionWidth <5), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidt <86), sum(ResolutionWidth + 87), sum(ResolutionWidth  + 88), sum(ResolutionWid < 87), sum(ResolutionWidth + 88), sum(ResolutionWidt h + 89) FROM hits_10m; -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+ -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -1 tuple (940.847ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidt <dth), sum(ResolutionWidth + 1), sum(ResolutionWidth  + 2), sum(ResolutionWidt <+ 1), sum(ResolutionWidth + 2), sum(ResolutionWidth  + 3), sum(ResolutionWidt <+ 2), sum(ResolutionWidth + 3), sum(ResolutionWidth  + 4), sum(ResolutionWidt <+ 3), sum(ResolutionWidth + 4), sum(ResolutionWidth  + 5), sum(ResolutionWidt <+ 4), sum(ResolutionWidth + 5), sum(ResolutionWidth  + 6), sum(ResolutionWidt <+ 5), sum(ResolutionWidth + 6), sum(ResolutionWidth  + 7), sum(ResolutionWidt <+ 6), sum(ResolutionWidth + 7), sum(ResolutionWidth  + 8), sum(ResolutionWidt <+ 7), sum(ResolutionWidth + 8), sum(ResolutionWidth  + 9), sum(ResolutionWidt <+ 8), sum(ResolutionWidth + 9), sum(ResolutionWidth  + 10), sum(ResolutionWid <+ 9), sum(ResolutionWidth + 10), sum(ResolutionWidt h + 11), sum(ResolutionWi <+ 10), sum(ResolutionWidth + 11), sum(ResolutionWid th + 12), sum(ResolutionW < + 11), sum(ResolutionWidth + 12), sum(ResolutionWi dth + 13), sum(Resolution <h + 12), sum(ResolutionWidth + 13), sum(ResolutionW idth + 14), sum(Resolutio <th + 13), sum(ResolutionWidth + 14), sum(Resolution Width + 15), sum(Resoluti <dth + 14), sum(ResolutionWidth + 15), sum(Resolutio nWidth + 16), sum(Resolut <idth + 15), sum(ResolutionWidth + 16), sum(Resoluti onWidth + 17), sum(Resolu <Width + 16), sum(ResolutionWidth + 17), sum(Resolut ionWidth + 18), sum(Resol <nWidth + 17), sum(ResolutionWidth + 18), sum(Resolu tionWidth + 19), sum(Reso <onWidth + 18), sum(ResolutionWidth + 19), sum(Resol utionWidth + 20), sum(Res <ionWidth + 19), sum(ResolutionWidth + 20), sum(Reso lutionWidth + 21), sum(Re <tionWidth + 20), sum(ResolutionWidth + 21), sum(Res olutionWidth + 22), sum(R <utionWidth + 21), sum(ResolutionWidth + 22), sum(Re solutionWidth + 23), sum( <lutionWidth + 22), sum(ResolutionWidth + 23), sum(R esolutionWidth + 24), sum <olutionWidth + 23), sum(ResolutionWidth + 24), sum( ResolutionWidth + 25), su <solutionWidth + 24), sum(ResolutionWidth + 25), sum (ResolutionWidth + 26), s <esolutionWidth + 25), sum(ResolutionWidth + 26), su m(ResolutionWidth + 27), <ResolutionWidth + 26), sum(ResolutionWidth + 27), s um(ResolutionWidth + 28), <(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29) <m(ResolutionWidth + 28), sum(ResolutionWidth + 29),  sum(ResolutionWidth + 30 <um(ResolutionWidth + 29), sum(ResolutionWidth + 30) , sum(ResolutionWidth + 3 <sum(ResolutionWidth + 30), sum(ResolutionWidth + 31 ), sum(ResolutionWidth + < sum(ResolutionWidth + 31), sum(ResolutionWidth + 3 2), sum(ResolutionWidth + <, sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth <), sum(ResolutionWidth + 33), sum(ResolutionWidth +  34), sum(ResolutionWidth <3), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidt <34), sum(ResolutionWidth + 35), sum(ResolutionWidth  + 36), sum(ResolutionWid < 35), sum(ResolutionWidth + 36), sum(ResolutionWidt h + 37), sum(ResolutionWi <+ 36), sum(ResolutionWidth + 37), sum(ResolutionWid th + 38), sum(ResolutionW < + 37), sum(ResolutionWidth + 38), sum(ResolutionWi dth + 39), sum(Resolution <h + 38), sum(ResolutionWidth + 39), sum(ResolutionW idth + 40), sum(Resolutio <th + 39), sum(ResolutionWidth + 40), sum(Resolution Width + 41), sum(Resoluti <dth + 40), sum(ResolutionWidth + 41), sum(Resolutio nWidth + 42), sum(Resolut <idth + 41), sum(ResolutionWidth + 42), sum(Resoluti onWidth + 43), sum(Resolu <Width + 42), sum(ResolutionWidth + 43), sum(Resolut ionWidth + 44), sum(Resol <nWidth + 43), sum(ResolutionWidth + 44), sum(Resolu tionWidth + 45), sum(Reso <onWidth + 44), sum(ResolutionWidth + 45), sum(Resol utionWidth + 46), sum(Res <ionWidth + 45), sum(ResolutionWidth + 46), sum(Reso lutionWidth + 47), sum(Re <tionWidth + 46), sum(ResolutionWidth + 47), sum(Res olutionWidth + 48), sum(R <utionWidth + 47), sum(ResolutionWidth + 48), sum(Re solutionWidth + 49), sum( <lutionWidth + 48), sum(ResolutionWidth + 49), sum(R esolutionWidth + 50), sum <olutionWidth + 49), sum(ResolutionWidth + 50), sum( ResolutionWidth + 51), su <solutionWidth + 50), sum(ResolutionWidth + 51), sum (ResolutionWidth + 52), s <esolutionWidth + 51), sum(ResolutionWidth + 52), su m(ResolutionWidth + 53), <ResolutionWidth + 52), sum(ResolutionWidth + 53), s um(ResolutionWidth + 54), <(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55) <m(ResolutionWidth + 54), sum(ResolutionWidth + 55),  sum(ResolutionWidth + 56 <um(ResolutionWidth + 55), sum(ResolutionWidth + 56) , sum(ResolutionWidth + 5 <sum(ResolutionWidth + 56), sum(ResolutionWidth + 57 ), sum(ResolutionWidth + < sum(ResolutionWidth + 57), sum(ResolutionWidth + 5 8), sum(ResolutionWidth + <, sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth <), sum(ResolutionWidth + 59), sum(ResolutionWidth +  60), sum(ResolutionWidth <9), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidt <60), sum(ResolutionWidth + 61), sum(ResolutionWidth  + 62), sum(ResolutionWid < 61), sum(ResolutionWidth + 62), sum(ResolutionWidt h + 63), sum(ResolutionWi <+ 62), sum(ResolutionWidth + 63), sum(ResolutionWid th + 64), sum(ResolutionW < + 63), sum(ResolutionWidth + 64), sum(ResolutionWi dth + 65), sum(Resolution <h + 64), sum(ResolutionWidth + 65), sum(ResolutionW idth + 66), sum(Resolutio <th + 65), sum(ResolutionWidth + 66), sum(Resolution Width + 67), sum(Resoluti <dth + 66), sum(ResolutionWidth + 67), sum(Resolutio nWidth + 68), sum(Resolut <idth + 67), sum(ResolutionWidth + 68), sum(Resoluti onWidth + 69), sum(Resolu <Width + 68), sum(ResolutionWidth + 69), sum(Resolut ionWidth + 70), sum(Resol <nWidth + 69), sum(ResolutionWidth + 70), sum(Resolu tionWidth + 71), sum(Reso <onWidth + 70), sum(ResolutionWidth + 71), sum(Resol utionWidth + 72), sum(Res <ionWidth + 71), sum(ResolutionWidth + 72), sum(Reso lutionWidth + 73), sum(Re <tionWidth + 72), sum(ResolutionWidth + 73), sum(Res olutionWidth + 74), sum(R <utionWidth + 73), sum(ResolutionWidth + 74), sum(Re solutionWidth + 75), sum( <lutionWidth + 74), sum(ResolutionWidth + 75), sum(R esolutionWidth + 76), sum <olutionWidth + 75), sum(ResolutionWidth + 76), sum( ResolutionWidth + 77), su <solutionWidth + 76), sum(ResolutionWidth + 77), sum (ResolutionWidth + 78), s <esolutionWidth + 77), sum(ResolutionWidth + 78), su m(ResolutionWidth + 79), <ResolutionWidth + 78), sum(ResolutionWidth + 79), s um(ResolutionWidth + 80), <(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81) <m(ResolutionWidth + 80), sum(ResolutionWidth + 81),  sum(ResolutionWidth + 82 <um(ResolutionWidth + 81), sum(ResolutionWidth + 82) , sum(ResolutionWidth + 8 <sum(ResolutionWidth + 82), sum(ResolutionWidth + 83 ), sum(ResolutionWidth + < sum(ResolutionWidth + 83), sum(ResolutionWidth + 8 4), sum(ResolutionWidth + <, sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth <), sum(ResolutionWidth + 85), sum(ResolutionWidth +  86), sum(ResolutionWidth <5), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidt <86), sum(ResolutionWidth + 87), sum(ResolutionWidth  + 88), sum(ResolutionWid < 87), sum(ResolutionWidth + 88), sum(ResolutionWidt h + 89) FROM hits_10m; -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+=============+ -+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+-------------+ -1 tuple (921.118ms) -sql>\q --- много тупых агрегатных функций.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(Resolut <ClientIP, count(*) AS c, sum(Refresh), avg(Resoluti onWidth) FROM hits_10m WH <um(Refresh), avg(ResolutionWidth) FROM hits_10m WHE RE SearchPhrase <> '' GRO <nWidth) FROM hits_10m WHERE SearchPhrase <> '' GROU P BY SearchEngineID, Clie <E SearchPhrase <> '' GROUP BY SearchEngineID, Clien tIP ORDER BY count(*) DES < BY SearchEngineID, ClientIP ORDER BY count(*) DESC  LIMIT 10; -+----------------+------------+------+------+--------------------------+ -+================+============+======+======+==========================+ -+----------------+------------+------+------+--------------------------+ -10 tuples (17.7s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(Resolut <ClientIP, count(*) AS c, sum(Refresh), avg(Resoluti onWidth) FROM hits_10m WH <um(Refresh), avg(ResolutionWidth) FROM hits_10m WHE RE SearchPhrase <> '' GRO <nWidth) FROM hits_10m WHERE SearchPhrase <> '' GROU P BY SearchEngineID, Clie <E SearchPhrase <> '' GROUP BY SearchEngineID, Clien tIP ORDER BY count(*) DES < BY SearchEngineID, ClientIP ORDER BY count(*) DESC  LIMIT 10; -+----------------+------------+------+------+--------------------------+ -+================+============+======+======+==========================+ -+----------------+------------+------+------+--------------------------+ -10 tuples (18.2s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(Resolut <ClientIP, count(*) AS c, sum(Refresh), avg(Resoluti onWidth) FROM hits_10m WH <um(Refresh), avg(ResolutionWidth) FROM hits_10m WHE RE SearchPhrase <> '' GRO <nWidth) FROM hits_10m WHERE SearchPhrase <> '' GROU P BY SearchEngineID, Clie <E SearchPhrase <> '' GROUP BY SearchEngineID, Clien tIP ORDER BY count(*) DES < BY SearchEngineID, ClientIP ORDER BY count(*) DESC  LIMIT 10; -+----------------+------------+------+------+--------------------------+ -+================+============+======+======+==========================+ -+----------------+------------+------+------+--------------------------+ -10 tuples (17.5s) -sql>\q --- сложная агрегация, для больших таблиц может не хватить оперативки.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidt <P, count(*) AS c, sum(Refresh), avg(ResolutionWidth ) FROM hits_10m WHERE Sea <esh), avg(ResolutionWidth) FROM hits_10m WHERE Sear chPhrase <> '' GROUP BY W < FROM hits_10m WHERE SearchPhrase <> '' GROUP BY Wa tchID, ClientIP ORDER BY <hPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c ount(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (2.0s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidt <P, count(*) AS c, sum(Refresh), avg(ResolutionWidth ) FROM hits_10m WHERE Sea <esh), avg(ResolutionWidth) FROM hits_10m WHERE Sear chPhrase <> '' GROUP BY W < FROM hits_10m WHERE SearchPhrase <> '' GROUP BY Wa tchID, ClientIP ORDER BY <hPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c ount(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (236.466ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidt <P, count(*) AS c, sum(Refresh), avg(ResolutionWidth ) FROM hits_10m WHERE Sea <esh), avg(ResolutionWidth) FROM hits_10m WHERE Sear chPhrase <> '' GROUP BY W < FROM hits_10m WHERE SearchPhrase <> '' GROUP BY Wa tchID, ClientIP ORDER BY <hPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c ount(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (236.223ms) -sql>\q --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidt <P, count(*) AS c, sum(Refresh), avg(ResolutionWidth ) FROM hits_10m GROUP BY <esh), avg(ResolutionWidth) FROM hits_10m GROUP BY W atchID, ClientIP ORDER BY < FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (1.3s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidt <P, count(*) AS c, sum(Refresh), avg(ResolutionWidth ) FROM hits_10m GROUP BY <esh), avg(ResolutionWidth) FROM hits_10m GROUP BY W atchID, ClientIP ORDER BY < FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (1.3s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidt <P, count(*) AS c, sum(Refresh), avg(ResolutionWidth ) FROM hits_10m GROUP BY <esh), avg(ResolutionWidth) FROM hits_10m GROUP BY W atchID, ClientIP ORDER BY < FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; -+---------------------+------------+------+------+--------------------------+ -+=====================+============+======+======+==========================+ -+---------------------+------------+------+------+--------------------------+ -10 tuples (1.3s) -sql>\q --- то же самое, но ещё и без фильтрации.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LI <OM hits_10m GROUP BY URL ORDER BY count(*) DESC LIM IT 10; -+--------------------------------------------------------------+--------+ -+==============================================================+========+ -+--------------------------------------------------------------+--------+ -10 tuples (2m 18s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LI <OM hits_10m GROUP BY URL ORDER BY count(*) DESC LIM IT 10; -+--------------------------------------------------------------+--------+ -+==============================================================+========+ -+--------------------------------------------------------------+--------+ -10 tuples (2m 20s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LI <OM hits_10m GROUP BY URL ORDER BY count(*) DESC LIM IT 10; -+--------------------------------------------------------------+--------+ -+==============================================================+========+ -+--------------------------------------------------------------+--------+ -10 tuples (2m 53s) -sql>\q --- агрегация по URL.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) D < FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DE SC LIMIT 10; -syntax error, unexpected sqlINT in: "select 1, url, count(*) from hits_10m group by 1" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) D < FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DE SC LIMIT 10; -syntax error, unexpected sqlINT in: "select 1, url, count(*) from hits_10m group by 1" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) D < FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DE SC LIMIT 10; -syntax error, unexpected sqlINT in: "select 1, url, count(*) from hits_10m group by 1" -sql>\q --- агрегация по URL и числу.; - -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM <IP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM h its_10m GROUP BY ClientIP <ntIP - 3, count(*) FROM hits_10m GROUP BY ClientIP,  ClientIP - 1, ClientIP - <ts_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY <ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c ount(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_10m group by clientip, clientip -" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM <IP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM h its_10m GROUP BY ClientIP <ntIP - 3, count(*) FROM hits_10m GROUP BY ClientIP,  ClientIP - 1, ClientIP - <ts_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY <ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c ount(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_10m group by clientip, clientip -" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM <IP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM h its_10m GROUP BY ClientIP <ntIP - 3, count(*) FROM hits_10m GROUP BY ClientIP,  ClientIP - 1, ClientIP - <ts_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY <ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c ount(*) DESC LIMIT 10; -syntax error, unexpected '-', expecting SCOLON in: "select clientip, clientip - 1, clientip - 2, clientip - 3, count(*) from hits_10m group by clientip, clientip -" -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 <*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07 <10m WHERE CounterID = 34 AND EventDate >= '2013-07- 01' AND EventDate <= '201 <ND EventDate >= '2013-07-01' AND EventDate <= '2013 -07-31' AND NOT DontCount <1' AND EventDate <= '2013-07-31' AND NOT DontCountH its AND NOT Refresh AND U <07-31' AND NOT DontCountHits AND NOT Refresh AND UR L <> '' GROUP BY URL ORDE <ts AND NOT Refresh AND URL <> '' GROUP BY URL ORDER  BY PageViews DESC LIMIT < <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 1 0; -+---------------------------------------------------------------------------------+-----------+ -+=================================================================================+===========+ -+---------------------------------------------------------------------------------+-----------+ -10 tuples (3m 9s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 <*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07 <10m WHERE CounterID = 34 AND EventDate >= '2013-07- 01' AND EventDate <= '201 <ND EventDate >= '2013-07-01' AND EventDate <= '2013 -07-31' AND NOT DontCount <1' AND EventDate <= '2013-07-31' AND NOT DontCountH its AND NOT Refresh AND U <07-31' AND NOT DontCountHits AND NOT Refresh AND UR L <> '' GROUP BY URL ORDE <ts AND NOT Refresh AND URL <> '' GROUP BY URL ORDER  BY PageViews DESC LIMIT < <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 1 0; -+---------------------------------------------------------------------------------+-----------+ -+=================================================================================+===========+ -+---------------------------------------------------------------------------------+-----------+ -10 tuples (2m 49s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 <*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07 <10m WHERE CounterID = 34 AND EventDate >= '2013-07- 01' AND EventDate <= '201 <ND EventDate >= '2013-07-01' AND EventDate <= '2013 -07-31' AND NOT DontCount <1' AND EventDate <= '2013-07-31' AND NOT DontCountH its AND NOT Refresh AND U <07-31' AND NOT DontCountHits AND NOT Refresh AND UR L <> '' GROUP BY URL ORDE <ts AND NOT Refresh AND URL <> '' GROUP BY URL ORDER  BY PageViews DESC LIMIT < <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 1 0; -+---------------------------------------------------------------------------------+-----------+ -+=================================================================================+===========+ -+---------------------------------------------------------------------------------+-----------+ -10 tuples (2m 55s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AN <AS PageViews FROM hits_10m WHERE CounterID = 34 AND  EventDate >= '2013-07-01 < WHERE CounterID = 34 AND EventDate >= '2013-07-01'  AND EventDate <= '2013-0 <EventDate >= '2013-07-01' AND EventDate <= '2013-07 -31' AND NOT DontCountHit <AND EventDate <= '2013-07-31' AND NOT DontCountHits  AND NOT Refresh AND Titl <31' AND NOT DontCountHits AND NOT Refresh AND Title  <> '' GROUP BY Title ORD <AND NOT Refresh AND Title <> '' GROUP BY Title ORDE R BY PageViews DESC LIMIT <<> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+------------------------------------------------+-----------+ -+================================================+===========+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AN <AS PageViews FROM hits_10m WHERE CounterID = 34 AND  EventDate >= '2013-07-01 < WHERE CounterID = 34 AND EventDate >= '2013-07-01'  AND EventDate <= '2013-0 <EventDate >= '2013-07-01' AND EventDate <= '2013-07 -31' AND NOT DontCountHit <AND EventDate <= '2013-07-31' AND NOT DontCountHits  AND NOT Refresh AND Titl <31' AND NOT DontCountHits AND NOT Refresh AND Title  <> '' GROUP BY Title ORD <AND NOT Refresh AND Title <> '' GROUP BY Title ORDE R BY PageViews DESC LIMIT <<> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+------------------------------------------------+-----------+ -+================================================+===========+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AN <AS PageViews FROM hits_10m WHERE CounterID = 34 AND  EventDate >= '2013-07-01 < WHERE CounterID = 34 AND EventDate >= '2013-07-01'  AND EventDate <= '2013-0 <EventDate >= '2013-07-01' AND EventDate <= '2013-07 -31' AND NOT DontCountHit <AND EventDate <= '2013-07-31' AND NOT DontCountHits  AND NOT Refresh AND Titl <31' AND NOT DontCountHits AND NOT Refresh AND Title  <> '' GROUP BY Title ORD <AND NOT Refresh AND Title <> '' GROUP BY Title ORDE R BY PageViews DESC LIMIT <<> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; -+------------------------------------------------+-----------+ -+================================================+===========+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND < PageViews FROM hits_10m WHERE CounterID = 34 AND E ventDate >= '2013-07-01' <HERE CounterID = 34 AND EventDate >= '2013-07-01' A ND EventDate <= '2013-07- <entDate >= '2013-07-01' AND EventDate <= '2013-07-3 1' AND NOT Refresh AND Is <D EventDate <= '2013-07-31' AND NOT Refresh AND IsL ink AND NOT IsDownload GR <' AND NOT Refresh AND IsLink AND NOT IsDownload GRO UP BY URL ORDER BY PageVi <nk AND NOT IsDownload GROUP BY URL ORDER BY PageVie ws DESC LIMIT 1000; -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+=========================================================================================================================================================================================================================================================================================================================================================================================================+===========+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND < PageViews FROM hits_10m WHERE CounterID = 34 AND E ventDate >= '2013-07-01' <HERE CounterID = 34 AND EventDate >= '2013-07-01' A ND EventDate <= '2013-07- <entDate >= '2013-07-01' AND EventDate <= '2013-07-3 1' AND NOT Refresh AND Is <D EventDate <= '2013-07-31' AND NOT Refresh AND IsL ink AND NOT IsDownload GR <' AND NOT Refresh AND IsLink AND NOT IsDownload GRO UP BY URL ORDER BY PageVi <nk AND NOT IsDownload GROUP BY URL ORDER BY PageVie ws DESC LIMIT 1000; -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+=========================================================================================================================================================================================================================================================================================================================================================================================================+===========+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND < PageViews FROM hits_10m WHERE CounterID = 34 AND E ventDate >= '2013-07-01' <HERE CounterID = 34 AND EventDate >= '2013-07-01' A ND EventDate <= '2013-07- <entDate >= '2013-07-01' AND EventDate <= '2013-07-3 1' AND NOT Refresh AND Is <D EventDate <= '2013-07-31' AND NOT Refresh AND IsL ink AND NOT IsDownload GR <' AND NOT Refresh AND IsLink AND NOT IsDownload GRO UP BY URL ORDER BY PageVi <nk AND NOT IsDownload GROUP BY URL ORDER BY PageVie ws DESC LIMIT 1000; -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+=========================================================================================================================================================================================================================================================================================================================================================================================================+===========+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngin <SearchEngineID, AdvEngineID, CASE WHEN SearchEngine ID = 0 AND AdvEngineID = <D, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0  THEN Referer ELSE '' EN <D = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END  AS Src, URL AS Dst, coun <THEN Referer ELSE '' END AS Src, URL AS Dst, count (*) AS PageViews FROM hit <AS Src, URL AS Dst, count(*) AS PageViews FROM hits _10m WHERE CounterID = 34 <*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07 <10m WHERE CounterID = 34 AND EventDate >= '2013-07- 01' AND EventDate <= '201 <ND EventDate >= '2013-07-01' AND EventDate <= '2013 -07-31' AND NOT Refresh G <1' AND EventDate <= '2013-07-31' AND NOT Refresh GR OUP BY TraficSourceID, Se <07-31' AND NOT Refresh GROUP BY TraficSourceID, Sea rchEngineID, AdvEngineID, <UP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageVie <chEngineID, AdvEngineID, Src, Dst ORDER BY PageView s DESC LIMIT 1000; -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+================+================+=============+===================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================+===================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================+===========+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngin <SearchEngineID, AdvEngineID, CASE WHEN SearchEngine ID = 0 AND AdvEngineID = <D, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0  THEN Referer ELSE '' EN <D = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END  AS Src, URL AS Dst, coun <THEN Referer ELSE '' END AS Src, URL AS Dst, count (*) AS PageViews FROM hit <AS Src, URL AS Dst, count(*) AS PageViews FROM hits _10m WHERE CounterID = 34 <*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07 <10m WHERE CounterID = 34 AND EventDate >= '2013-07- 01' AND EventDate <= '201 <ND EventDate >= '2013-07-01' AND EventDate <= '2013 -07-31' AND NOT Refresh G <1' AND EventDate <= '2013-07-31' AND NOT Refresh GR OUP BY TraficSourceID, Se <07-31' AND NOT Refresh GROUP BY TraficSourceID, Sea rchEngineID, AdvEngineID, <UP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageVie <chEngineID, AdvEngineID, Src, Dst ORDER BY PageView s DESC LIMIT 1000; -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+================+================+=============+===================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================+===================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================+===========+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngin <SearchEngineID, AdvEngineID, CASE WHEN SearchEngine ID = 0 AND AdvEngineID = <D, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0  THEN Referer ELSE '' EN <D = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END  AS Src, URL AS Dst, coun <THEN Referer ELSE '' END AS Src, URL AS Dst, count (*) AS PageViews FROM hit <AS Src, URL AS Dst, count(*) AS PageViews FROM hits _10m WHERE CounterID = 34 <*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07 <10m WHERE CounterID = 34 AND EventDate >= '2013-07- 01' AND EventDate <= '201 <ND EventDate >= '2013-07-01' AND EventDate <= '2013 -07-31' AND NOT Refresh G <1' AND EventDate <= '2013-07-31' AND NOT Refresh GR OUP BY TraficSourceID, Se <07-31' AND NOT Refresh GROUP BY TraficSourceID, Sea rchEngineID, AdvEngineID, <UP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageVie <chEngineID, AdvEngineID, Src, Dst ORDER BY PageView s DESC LIMIT 1000; -+----------------+----------------+-------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ -+================+================+=============+===================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================+===================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================================+===========+ -write error -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE Coun <te, count(*) AS PageViews FROM hits_10m WHERE Count erID = 34 AND EventDate > <FROM hits_10m WHERE CounterID = 34 AND EventDate >=  '2013-07-01' AND EventDa <rID = 34 AND EventDate >= '2013-07-01' AND EventDat e <= '2013-07-31' AND NOT <'2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceI < <= '2013-07-31' AND NOT Refresh AND TraficSourceID  IN (-1, 6) AND RefererHa <efresh AND TraficSourceID IN (-1, 6) AND RefererHas h = 6202628419148573758 <IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDa < = 6202628419148573758 GROUP BY URLHash, EventDat e ORDER BY PageViews DESC <ROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (524.173ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE Coun <te, count(*) AS PageViews FROM hits_10m WHERE Count erID = 34 AND EventDate > <FROM hits_10m WHERE CounterID = 34 AND EventDate >=  '2013-07-01' AND EventDa <rID = 34 AND EventDate >= '2013-07-01' AND EventDat e <= '2013-07-31' AND NOT <'2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceI < <= '2013-07-31' AND NOT Refresh AND TraficSourceID  IN (-1, 6) AND RefererHa <efresh AND TraficSourceID IN (-1, 6) AND RefererHas h = 6202628419148573758 <IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDa < = 6202628419148573758 GROUP BY URLHash, EventDat e ORDER BY PageViews DESC <ROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (66.989ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE Coun <te, count(*) AS PageViews FROM hits_10m WHERE Count erID = 34 AND EventDate > <FROM hits_10m WHERE CounterID = 34 AND EventDate >=  '2013-07-01' AND EventDa <rID = 34 AND EventDate >= '2013-07-01' AND EventDat e <= '2013-07-31' AND NOT <'2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceI < <= '2013-07-31' AND NOT Refresh AND TraficSourceID  IN (-1, 6) AND RefererHa <efresh AND TraficSourceID IN (-1, 6) AND RefererHas h = 6202628419148573758 <IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDa < = 6202628419148573758 GROUP BY URLHash, EventDat e ORDER BY PageViews DESC <ROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; -+---------+-----------+-----------+ -+=========+===========+===========+ -+---------+-----------+-----------+ -0 tuples (123.604ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM <h, WindowClientHeight, count(*) AS PageViews FROM h its_10m WHERE CounterID = <nt(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013 <ts_10m WHERE CounterID = 34 AND EventDate >= '2013- 07-01' AND EventDate <= ' <4 AND EventDate >= '2013-07-01' AND EventDate <= '2 013-07-31' AND NOT Refres <7-01' AND EventDate <= '2013-07-31' AND NOT Refresh  AND NOT DontCountHits AN <13-07-31' AND NOT Refresh AND NOT DontCountHits AND  URLHash = 6202628419148 <AND NOT DontCountHits AND URLHash = 62026284191485 73758 GROUP BY WindowClie <URLHash = 6202628419148573758 GROUP BY WindowClien tWidth, WindowClientHeigh <3758 GROUP BY WindowClientWidth, WindowClientHeight  ORDER BY PageViews DESC <Width, WindowClientHeight ORDER BY PageViews DESC L IMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (305.343ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM <h, WindowClientHeight, count(*) AS PageViews FROM h its_10m WHERE CounterID = <nt(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013 <ts_10m WHERE CounterID = 34 AND EventDate >= '2013- 07-01' AND EventDate <= ' <4 AND EventDate >= '2013-07-01' AND EventDate <= '2 013-07-31' AND NOT Refres <7-01' AND EventDate <= '2013-07-31' AND NOT Refresh  AND NOT DontCountHits AN <13-07-31' AND NOT Refresh AND NOT DontCountHits AND  URLHash = 6202628419148 <AND NOT DontCountHits AND URLHash = 62026284191485 73758 GROUP BY WindowClie <URLHash = 6202628419148573758 GROUP BY WindowClien tWidth, WindowClientHeigh <3758 GROUP BY WindowClientWidth, WindowClientHeight  ORDER BY PageViews DESC <Width, WindowClientHeight ORDER BY PageViews DESC L IMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (30.736ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM <h, WindowClientHeight, count(*) AS PageViews FROM h its_10m WHERE CounterID = <nt(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013 <ts_10m WHERE CounterID = 34 AND EventDate >= '2013- 07-01' AND EventDate <= ' <4 AND EventDate >= '2013-07-01' AND EventDate <= '2 013-07-31' AND NOT Refres <7-01' AND EventDate <= '2013-07-31' AND NOT Refresh  AND NOT DontCountHits AN <13-07-31' AND NOT Refresh AND NOT DontCountHits AND  URLHash = 6202628419148 <AND NOT DontCountHits AND URLHash = 62026284191485 73758 GROUP BY WindowClie <URLHash = 6202628419148573758 GROUP BY WindowClien tWidth, WindowClientHeigh <3758 GROUP BY WindowClientWidth, WindowClientHeight  ORDER BY PageViews DESC <Width, WindowClientHeight ORDER BY PageViews DESC L IMIT 10000; -+-------------------+--------------------+-----------+ -+===================+====================+===========+ -+-------------------+--------------------+-----------+ -0 tuples (48.725ms) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 1 -query: SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS Page <act (SECOND from EventTime) AS M, count(*) AS PageV iews FROM hits_10m WHERE <) AS M, count(*) AS PageViews FROM hits_10m WHERE C ounterID = 34 AND EventDa <ews FROM hits_10m WHERE CounterID = 34 AND EventDat e >= '2013-07-01' <unterID = 34 AND EventDate >= '2013-07-01'  AND Event < >= '2013-07-01' AND EventD ate <= '2013-07-02' AND N < AND EventDate <= '2013-07-02' AND NO T Refresh AND NOT DontCou <te <= '2013-07-02' AND NOT Refresh AND NOT DontCoun tHits GROUP BY M ORDER BY < Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -+----------------------------+-----------+ -+============================+===========+ -+----------------------------+-----------+ -652 tuples (2.3s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 2 -query: SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS Page <act (SECOND from EventTime) AS M, count(*) AS PageV iews FROM hits_10m WHERE <) AS M, count(*) AS PageViews FROM hits_10m WHERE C ounterID = 34 AND EventDa <ews FROM hits_10m WHERE CounterID = 34 AND EventDat e >= '2013-07-01' <unterID = 34 AND EventDate >= '2013-07-01'  AND Event < >= '2013-07-01' AND EventD ate <= '2013-07-02' AND N < AND EventDate <= '2013-07-02' AND NO T Refresh AND NOT DontCou <te <= '2013-07-02' AND NOT Refresh AND NOT DontCoun tHits GROUP BY M ORDER BY < Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -+----------------------------+-----------+ -+============================+===========+ -+----------------------------+-----------+ -652 tuples (1.9s) -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q - -times: 3 -query: SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_10m WHERE CounterID = 34 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS Page <act (SECOND from EventTime) AS M, count(*) AS PageV iews FROM hits_10m WHERE <) AS M, count(*) AS PageViews FROM hits_10m WHERE C ounterID = 34 AND EventDa <ews FROM hits_10m WHERE CounterID = 34 AND EventDat e >= '2013-07-01' <unterID = 34 AND EventDate >= '2013-07-01'  AND Event < >= '2013-07-01' AND EventD ate <= '2013-07-02' AND N < AND EventDate <= '2013-07-02' AND NO T Refresh AND NOT DontCou <te <= '2013-07-02' AND NOT Refresh AND NOT DontCoun tHits GROUP BY M ORDER BY < Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; -+----------------------------+-----------+ -+============================+===========+ -+----------------------------+-----------+ -652 tuples (1.9s) -sql>\q -stop time: Tue Sep 17 10:52:42 MSK 2013 diff --git a/benchmark/monetdb/log/log_upload_100m b/benchmark/monetdb/log/log_upload_100m deleted file mode 100644 index d59f8bd4b70..00000000000 --- a/benchmark/monetdb/log/log_upload_100m +++ /dev/null @@ -1,62 +0,0 @@ -start time: Thu Sep 12 06:33:47 MSK 2013 -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: create table hits_100m ( WatchID BIGINT, JavaEnable SMALLINT, Title VARCHAR(1400), GoodEvent SMALLINT, EventTime TIMESTAMP, EventDate DATE, CounterID BIGINT, ClientIP BIGINT, RegionID BIGINT, UserID BIGINT, CounterClass TINYINT, OS SMALLINT, UserAgent SMALLINT, URL VARCHAR(7800), Referer VARCHAR(3125), Refresh TINYINT, RefererCategoryID INT, RefererRegionID BIGINT, URLCategoryID INT, URLRegionID BIGINT, ResolutionWidth INT, ResolutionHeight INT, ResolutionDepth SMALLINT, FlashMajor SMALLINT, FlashMinor SMALLINT, FlashMinor2 VARCHAR(256), NetMajor SMALLINT, NetMinor SMALLINT, UserAgentMajor INT, UserAgentMinor CHAR(2), CookieEnable SMALLINT, JavascriptEnable SMALLINT, IsMobile SMALLINT, MobilePhone SMALLINT, MobilePhoneModel VARCHAR(80), Params VARCHAR(2925), IPNetworkID BIGINT, TraficSourceID SMALLINT, SearchEngineID INT, SearchPhrase VARCHAR(2008), AdvEngineID SMALLINT, IsArtifical SMALLINT, WindowClientWidth INT, WindowClientHeight INT, ClientTimeZone INTEGER, ClientEventTime TIMESTAMP, SilverlightVersion1 SMALLINT, SilverlightVersion2 SMALLINT, SilverlightVersion3 BIGINT, SilverlightVersion4 INT, PageCharset VARCHAR(80), CodeVersion BIGINT, IsLink SMALLINT, IsDownload SMALLINT, IsNotBounce SMALLINT, FUniqID BIGINT, OriginalURL VARCHAR(8181), HID BIGINT, IsOldCounter SMALLINT, IsEvent SMALLINT, IsParameter SMALLINT, DontCountHits SMALLINT, WithHash SMALLINT, HitColor CHAR(1), LocalEventTime TIMESTAMP, Age SMALLINT, Sex SMALLINT, Income SMALLINT, Interests INT, Robotness SMALLINT, RemoteIP BIGINT, WindowName INT, OpenerName INT, HistoryLength SMALLINT, BrowserLanguage CHAR(2), BrowserCountry CHAR(2), SocialNetwork VARCHAR(128), SocialAction VARCHAR(128), HTTPError INT, SendTiming BIGINT, DNSTiming BIGINT, ConnectTiming BIGINT, ResponseStartTiming BIGINT, ResponseEndTiming BIGINT, FetchTiming BIGINT, SocialSourceNetworkID SMALLINT, SocialSourcePage VARCHAR(256), ParamPrice BIGINT, ParamOrderID VARCHAR(80), ParamCurrency CHAR(3), ParamCurrencyID INT, OpenstatServiceName VARCHAR(80), OpenstatCampaignID VARCHAR(512), OpenstatAdID VARCHAR(80), OpenstatSourceID VARCHAR(256), UTMSource VARCHAR(256), UTMMedium VARCHAR(256), UTMCampaign VARCHAR(407), UTMContent VARCHAR(256), UTMTerm VARCHAR(437), FromTag VARCHAR(428), HasGCLID SMALLINT, RefererHash BIGINT, URLHash BIGINT, CLID BIGINT, UserIDHash BIGINT ); CREATE INDEX hits_100m_ind ON hits_100m(CounterID, EventDate, UserIDHash, EventTime); -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>create table hits_100m ( WatchID BIGINT, JavaEnable SMALLINT, <( WatchID BIGINT, JavaEnable SMALLINT, Title VARCHAR(1400), <JavaEnable SMALLINT, Title VARCHAR(1400), G oodEvent SMALLINT, Ev <itle VARCHAR(1400), GoodEvent SMALLINT, Eve ntTime TIMESTAMP, Eve <odEvent SMALLINT, EventTime TIMESTAMP, Even tDate DATE, CounterID <tTime TIMESTAMP, EventDate DATE, CounterID BIGINT, ClientIP BIGI <Date DATE, CounterID BIGINT, ClientIP BIGIN T, RegionID BIGINT, <IGINT, ClientIP BIGINT, RegionID BIGINT,  UserID BIGINT, Coun <, RegionID BIGINT, UserID BIGINT, Count erClass TINYINT, OS S < UserID BIGINT, CounterClass TINYINT, OS SM ALLINT, UserAgent SMA <rClass TINYINT, OS SMALLINT, UserAgent SMAL LINT, URL VARCHAR(780 <LLINT, UserAgent SMALLINT, URL VARCHAR(7800 ), Referer VARCHAR(31 <INT, URL VARCHAR(7800), Referer VARCHAR(312 5), Refresh TINYINT, <, Referer VARCHAR(3125), Refresh TINYINT,  RefererCategoryID INT, <), Refresh TINYINT, RefererCategoryID INT,  RefererRegionID BIGIN < RefererCategoryID INT, RefererRegionID BIGINT , URLCategoryID INT, < RefererRegionID BIGINT, URLCategoryID INT,  URLRegionID BIGINT, < URLCategoryID INT, URLRegionID BIGINT,  ResolutionWidth INT, < URLRegionID BIGINT, ResolutionWidth INT, ResolutionHeight INT, <ResolutionWidth INT, ResolutionHeight INT, ResolutionDepth SMALLINT, <esolutionHeight INT, ResolutionDepth SMALLINT,  FlashMajor SMALLINT, <esolutionDepth SMALLINT, FlashMajor SMALLINT,  FlashMinor SMALLINT, < FlashMajor SMALLINT, FlashMinor SMALLINT,  FlashMinor2 VARCHAR(256 < FlashMinor SMALLINT, FlashMinor2 VARCHAR(256) , NetMajor SMALLINT, < FlashMinor2 VARCHAR(256), NetMajor SMALLINT,  NetMinor SMALLINT, < NetMajor SMALLINT, NetMinor SMALLINT, UserAgentMajor INT, U < NetMinor SMALLINT, UserAgentMajor INT, Us erAgentMinor CHAR(2), <serAgentMajor INT, UserAgentMinor CHAR(2), CookieEnable SMALLINT, <rAgentMinor CHAR(2), CookieEnable SMALLINT,  JavascriptEnable SMALLIN <ookieEnable SMALLINT, JavascriptEnable SMALLINT , IsMobile SMALLINT, <JavascriptEnable SMALLINT, IsMobile SMALLINT,  MobilePhone SMALLINT, < IsMobile SMALLINT, MobilePhone SMALLINT,  MobilePhoneModel VARCH < MobilePhone SMALLINT, MobilePhoneModel VARCHA R(80), Params VARCHAR < MobilePhoneModel VARCHAR(80), Params VARCHAR( 2925), IPNetworkID BI <(80), Params VARCHAR(2925), IPNetworkID BIG INT, TraficSourceID S <925), IPNetworkID BIGINT, TraficSourceID SM ALLINT, SearchEngineI <NT, TraficSourceID SMALLINT, SearchEngineID  INT, SearchPhrase VA <LLINT, SearchEngineID INT, SearchPhrase VAR CHAR(2008), AdvEngine <INT, SearchPhrase VARCHAR(2008), AdvEngineI D SMALLINT, IsArtific <HAR(2008), AdvEngineID SMALLINT, IsArtifica l SMALLINT, WindowCli < SMALLINT, IsArtifical SMALLINT, WindowClie ntWidth INT, WindowCl < SMALLINT, WindowClientWidth INT, WindowCli entHeight INT, Client <tWidth INT, WindowClientHeight INT, ClientT imeZone INTEGER, Clie <ntHeight INT, ClientTimeZone INTEGER, Clien tEventTime TIMESTAMP, <meZone INTEGER, ClientEventTime TIMESTAMP, SilverlightVersion1 SMALL <EventTime TIMESTAMP, SilverlightVersion1 SMALLI NT, SilverlightVersio <ilverlightVersion1 SMALLINT, SilverlightVersion 2 SMALLINT, Silverlig <T, SilverlightVersion2 SMALLINT, Silverligh tVersion3 BIGINT, Sil < SMALLINT, SilverlightVersion3 BIGINT, Silv erlightVersion4 INT, <Version3 BIGINT, SilverlightVersion4 INT, P ageCharset VARCHAR(80), <rlightVersion4 INT, PageCharset VARCHAR(80),  CodeVersion BIGINT, <geCharset VARCHAR(80), CodeVersion BIGINT, IsLink SMALLINT, IsDo < CodeVersion BIGINT, IsLink SMALLINT, IsDow nload SMALLINT, IsNot <sLink SMALLINT, IsDownload SMALLINT, IsNotB ounce SMALLINT, FUniq <load SMALLINT, IsNotBounce SMALLINT, FUniqI D BIGINT, OriginalURL <unce SMALLINT, FUniqID BIGINT, OriginalURL VARCHAR(8181), HID BI < BIGINT, OriginalURL VARCHAR(8181), HID BIG INT, IsOldCounter SMA <ARCHAR(8181), HID BIGINT, IsOldCounter SMAL LINT, IsEvent SMALLIN <NT, IsOldCounter SMALLINT, IsEvent SMALLINT , IsParameter SMALLIN <INT, IsEvent SMALLINT, IsParameter SMALLINT , DontCountHits SMALL < IsParameter SMALLINT, DontCountHits SMALLI NT, WithHash SMALLINT < DontCountHits SMALLINT, WithHash SMALLINT,  HitColor CHAR(1), <T, WithHash SMALLINT, HitColor CHAR(1),  LocalEventTime TIMESTAMP < HitColor CHAR(1), LocalEventTime TIMESTAMP,  Age SMALLINT, Se <LocalEventTime TIMESTAMP, Age SMALLINT, Sex  SMALLINT, Income SMA < Age SMALLINT, Sex SMALLINT, Income SMAL LINT, Interests INT, <SMALLINT, Income SMALLINT, Interests INT,  Robotness SMALLINT, <INT, Interests INT, Robotness SMALLINT,  RemoteIP BIGINT, Win < Robotness SMALLINT, RemoteIP BIGINT, Wind owName INT, OpenerNam <RemoteIP BIGINT, WindowName INT, OpenerName  INT, HistoryLength S <wName INT, OpenerName INT, HistoryLength SM ALLINT, BrowserLangua <INT, HistoryLength SMALLINT, BrowserLanguag e CHAR(2), BrowserCou <LLINT, BrowserLanguage CHAR(2), BrowserCoun try CHAR(2), SocialNe < CHAR(2), BrowserCountry CHAR(2), SocialNet work VARCHAR(128), So <ry CHAR(2), SocialNetwork VARCHAR(128), Soc ialAction VARCHAR(128), <ork VARCHAR(128), SocialAction VARCHAR(128),  HTTPError INT, Send <alAction VARCHAR(128), HTTPError INT, SendT iming BIGINT, DNSTimi < HTTPError INT, SendTiming BIGINT, DNSTimin g BIGINT, ConnectTimi <ming BIGINT, DNSTiming BIGINT, ConnectTimin g BIGINT, ResponseSta < BIGINT, ConnectTiming BIGINT, ResponseStar tTiming BIGINT, Respo < BIGINT, ResponseStartTiming BIGINT, Respon seEndTiming BIGINT, F <Timing BIGINT, ResponseEndTiming BIGINT, Fe tchTiming BIGINT, Soc <eEndTiming BIGINT, FetchTiming BIGINT, Soci alSourceNetworkID SMALLIN <chTiming BIGINT, SocialSourceNetworkID SMALLINT , SocialSourcePage VA <lSourceNetworkID SMALLINT, SocialSourcePage VAR CHAR(256), ParamPrice < SocialSourcePage VARCHAR(256), ParamPrice BIGINT, ParamOrderID <HAR(256), ParamPrice BIGINT, ParamOrderID V ARCHAR(80), ParamCurr <IGINT, ParamOrderID VARCHAR(80), ParamCurre ncy CHAR(3), ParamCur <RCHAR(80), ParamCurrency CHAR(3), ParamCurr encyID INT, OpenstatS <cy CHAR(3), ParamCurrencyID INT, OpenstatSe rviceName VARCHAR(80), <ncyID INT, OpenstatServiceName VARCHAR(80),  OpenstatCampaignID VARCH <viceName VARCHAR(80), OpenstatCampaignID VARCHA R(512), OpenstatAdID <OpenstatCampaignID VARCHAR(512), OpenstatAdID V ARCHAR(80), OpenstatS <(512), OpenstatAdID VARCHAR(80), OpenstatSo urceID VARCHAR(256), <RCHAR(80), OpenstatSourceID VARCHAR(256), U TMSource VARCHAR(256), <rceID VARCHAR(256), UTMSource VARCHAR(256),  UTMMedium VARCHAR(256), <MSource VARCHAR(256), UTMMedium VARCHAR(256),  UTMCampaign VARCHAR(40 <UTMMedium VARCHAR(256), UTMCampaign VARCHAR(407 ), UTMContent VARCHAR < UTMCampaign VARCHAR(407), UTMContent VARCHAR( 256), UTMTerm VARCHAR <, UTMContent VARCHAR(256), UTMTerm VARCHAR( 437), FromTag VARCHAR <56), UTMTerm VARCHAR(437), FromTag VARCHAR( 428), HasGCLID SMALLI <37), FromTag VARCHAR(428), HasGCLID SMALLIN T, RefererHash BIGINT <28), HasGCLID SMALLINT, RefererHash BIGINT,  URLHash BIGINT, <, RefererHash BIGINT, URLHash BIGINT, C LID BIGINT, UserIDHa < URLHash BIGINT, CLID BIGINT, UserIDHas h BIGINT ); CREATE INDEX <ID BIGINT, UserIDHash BIGINT ); CREATE INDEX hits_100m_ind ON hits_10 < BIGINT ); CREATE INDEX hits_100m_ind ON hits_100 m(CounterID, EventDate, U <its_100m_ind ON hits_100m(CounterID, EventDate, Us erIDHash, EventTime); -CREATE TABLE: name 'hits_100m' already in use -CREATE INDEX: name 'hits_100m_ind' already in use -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: COPY INTO hits_100m FROM ('/opt/dump/dump_0.3/dump_hits_100m_meshed_utf8.tsv') DELIMITERS '\t'; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>COPY INTO hits_100m FROM ('/opt/dump/dump_0.3/dump_hits_100m_meshed_utf8. <M ('/opt/dump/dump_0.3/dump_hits_100m_meshed_utf8.t sv') DELIMITERS '\t'; -\q -stop time: Thu Sep 12 06:55:23 MSK 2013 diff --git a/benchmark/monetdb/log/log_upload_1b b/benchmark/monetdb/log/log_upload_1b deleted file mode 100644 index 2bb5b8d2657..00000000000 --- a/benchmark/monetdb/log/log_upload_1b +++ /dev/null @@ -1,62 +0,0 @@ -start time: Thu Sep 12 08:31:54 MSK 2013 -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: create table hits_1b ( WatchID BIGINT, JavaEnable SMALLINT, Title VARCHAR(1400), GoodEvent SMALLINT, EventTime TIMESTAMP, EventDate DATE, CounterID BIGINT, ClientIP BIGINT, RegionID BIGINT, UserID BIGINT, CounterClass TINYINT, OS SMALLINT, UserAgent SMALLINT, URL VARCHAR(7800), Referer VARCHAR(3125), Refresh TINYINT, RefererCategoryID INT, RefererRegionID BIGINT, URLCategoryID INT, URLRegionID BIGINT, ResolutionWidth INT, ResolutionHeight INT, ResolutionDepth SMALLINT, FlashMajor SMALLINT, FlashMinor SMALLINT, FlashMinor2 VARCHAR(256), NetMajor SMALLINT, NetMinor SMALLINT, UserAgentMajor INT, UserAgentMinor CHAR(2), CookieEnable SMALLINT, JavascriptEnable SMALLINT, IsMobile SMALLINT, MobilePhone SMALLINT, MobilePhoneModel VARCHAR(80), Params VARCHAR(2925), IPNetworkID BIGINT, TraficSourceID SMALLINT, SearchEngineID INT, SearchPhrase VARCHAR(2008), AdvEngineID SMALLINT, IsArtifical SMALLINT, WindowClientWidth INT, WindowClientHeight INT, ClientTimeZone INTEGER, ClientEventTime TIMESTAMP, SilverlightVersion1 SMALLINT, SilverlightVersion2 SMALLINT, SilverlightVersion3 BIGINT, SilverlightVersion4 INT, PageCharset VARCHAR(80), CodeVersion BIGINT, IsLink SMALLINT, IsDownload SMALLINT, IsNotBounce SMALLINT, FUniqID BIGINT, OriginalURL VARCHAR(8181), HID BIGINT, IsOldCounter SMALLINT, IsEvent SMALLINT, IsParameter SMALLINT, DontCountHits SMALLINT, WithHash SMALLINT, HitColor CHAR(1), LocalEventTime TIMESTAMP, Age SMALLINT, Sex SMALLINT, Income SMALLINT, Interests INT, Robotness SMALLINT, RemoteIP BIGINT, WindowName INT, OpenerName INT, HistoryLength SMALLINT, BrowserLanguage CHAR(2), BrowserCountry CHAR(2), SocialNetwork VARCHAR(128), SocialAction VARCHAR(128), HTTPError INT, SendTiming BIGINT, DNSTiming BIGINT, ConnectTiming BIGINT, ResponseStartTiming BIGINT, ResponseEndTiming BIGINT, FetchTiming BIGINT, SocialSourceNetworkID SMALLINT, SocialSourcePage VARCHAR(256), ParamPrice BIGINT, ParamOrderID VARCHAR(80), ParamCurrency CHAR(3), ParamCurrencyID INT, OpenstatServiceName VARCHAR(80), OpenstatCampaignID VARCHAR(512), OpenstatAdID VARCHAR(80), OpenstatSourceID VARCHAR(256), UTMSource VARCHAR(256), UTMMedium VARCHAR(256), UTMCampaign VARCHAR(407), UTMContent VARCHAR(256), UTMTerm VARCHAR(437), FromTag VARCHAR(428), HasGCLID SMALLINT, RefererHash BIGINT, URLHash BIGINT, CLID BIGINT, UserIDHash BIGINT ); CREATE INDEX hits_1b_ind ON hits_1b(CounterID, EventDate, UserIDHash, EventTime); -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>create table hits_1b ( WatchID BIGINT, JavaEnable SMALLINT, T < WatchID BIGINT, JavaEnable SMALLINT, Ti tle VARCHAR(1400), Go <vaEnable SMALLINT, Title VARCHAR(1400), Goo dEvent SMALLINT, Even <le VARCHAR(1400), GoodEvent SMALLINT, Event Time TIMESTAMP, Event <Event SMALLINT, EventTime TIMESTAMP, EventD ate DATE, CounterID B <ime TIMESTAMP, EventDate DATE, CounterID BI GINT, ClientIP BIGINT <te DATE, CounterID BIGINT, ClientIP BIGINT,  RegionID BIGINT, <INT, ClientIP BIGINT, RegionID BIGINT, UserID BIGINT, Counte < RegionID BIGINT, UserID BIGINT, Counter Class TINYINT, OS SMA <serID BIGINT, CounterClass TINYINT, OS SMAL LINT, UserAgent SMALL <lass TINYINT, OS SMALLINT, UserAgent SMALLI NT, URL VARCHAR(7800) <INT, UserAgent SMALLINT, URL VARCHAR(7800),  Referer VARCHAR(3125 <T, URL VARCHAR(7800), Referer VARCHAR(3125) , Refresh TINYINT, < Referer VARCHAR(3125), Refresh TINYINT,  RefererCategoryID INT, < Refresh TINYINT, RefererCategoryID INT,  RefererRegionID BIGINT, <RefererCategoryID INT, RefererRegionID BIGINT,  URLCategoryID INT, < RefererRegionID BIGINT, URLCategoryID INT,  URLRegionID BIGINT, < URLCategoryID INT, URLRegionID BIGINT, R esolutionWidth INT, R <URLRegionID BIGINT, ResolutionWidth INT, Re solutionHeight INT, R <solutionWidth INT, ResolutionHeight INT, Re solutionDepth SMALLINT, <olutionHeight INT, ResolutionDepth SMALLINT,  FlashMajor SMALLINT, <olutionDepth SMALLINT, FlashMajor SMALLINT,  FlashMinor SMALLINT, < FlashMajor SMALLINT, FlashMinor SMALLINT, FlashMinor2 VARCHAR(256), <FlashMinor SMALLINT, FlashMinor2 VARCHAR(256),  NetMajor SMALLINT, <lashMinor2 VARCHAR(256), NetMajor SMALLINT,  NetMinor SMALLINT, U < NetMajor SMALLINT, NetMinor SMALLINT, Us erAgentMajor INT, Use <NetMinor SMALLINT, UserAgentMajor INT, User AgentMinor CHAR(2), C <rAgentMajor INT, UserAgentMinor CHAR(2), Co okieEnable SMALLINT, <gentMinor CHAR(2), CookieEnable SMALLINT, J avascriptEnable SMALLINT, <kieEnable SMALLINT, JavascriptEnable SMALLINT,  IsMobile SMALLINT, <vascriptEnable SMALLINT, IsMobile SMALLINT,  MobilePhone SMALLINT, < IsMobile SMALLINT, MobilePhone SMALLINT,  MobilePhoneModel VARCHAR <MobilePhone SMALLINT, MobilePhoneModel VARCHAR( 80), Params VARCHAR(2 <MobilePhoneModel VARCHAR(80), Params VARCHAR(29 25), IPNetworkID BIGI <0), Params VARCHAR(2925), IPNetworkID BIGIN T, TraficSourceID SMA <5), IPNetworkID BIGINT, TraficSourceID SMAL LINT, SearchEngineID <, TraficSourceID SMALLINT, SearchEngineID I NT, SearchPhrase VARC <INT, SearchEngineID INT, SearchPhrase VARCH AR(2008), AdvEngineID <T, SearchPhrase VARCHAR(2008), AdvEngineID SMALLINT, IsArtifical <R(2008), AdvEngineID SMALLINT, IsArtifical SMALLINT, WindowClien <MALLINT, IsArtifical SMALLINT, WindowClient Width INT, WindowClie <MALLINT, WindowClientWidth INT, WindowClien tHeight INT, ClientTi <idth INT, WindowClientHeight INT, ClientTim eZone INTEGER, Client <Height INT, ClientTimeZone INTEGER, ClientE ventTime TIMESTAMP, S <Zone INTEGER, ClientEventTime TIMESTAMP, Si lverlightVersion1 SMALLIN <entTime TIMESTAMP, SilverlightVersion1 SMALLINT , SilverlightVersion2 <verlightVersion1 SMALLINT, SilverlightVersion2 SMALLINT, Silverlight < SilverlightVersion2 SMALLINT, SilverlightV ersion3 BIGINT, Silve <MALLINT, SilverlightVersion3 BIGINT, Silver lightVersion4 INT, Pa <rsion3 BIGINT, SilverlightVersion4 INT, Pag eCharset VARCHAR(80), <ightVersion4 INT, PageCharset VARCHAR(80), CodeVersion BIGINT, I <Charset VARCHAR(80), CodeVersion BIGINT, Is Link SMALLINT, IsDown <odeVersion BIGINT, IsLink SMALLINT, IsDownl oad SMALLINT, IsNotBo <ink SMALLINT, IsDownload SMALLINT, IsNotBou nce SMALLINT, FUniqID <ad SMALLINT, IsNotBounce SMALLINT, FUniqID BIGINT, OriginalURL V <ce SMALLINT, FUniqID BIGINT, OriginalURL VA RCHAR(8181), HID BIGI <IGINT, OriginalURL VARCHAR(8181), HID BIGIN T, IsOldCounter SMALL <CHAR(8181), HID BIGINT, IsOldCounter SMALLI NT, IsEvent SMALLINT, <, IsOldCounter SMALLINT, IsEvent SMALLINT,  IsParameter SMALLINT, <T, IsEvent SMALLINT, IsParameter SMALLINT,  DontCountHits SMALLIN < IsParameter SMALLINT, DontCountHits SMALLINT , WithHash SMALLINT, < DontCountHits SMALLINT, WithHash SMALLINT,  HitColor CHAR(1), < WithHash SMALLINT, HitColor CHAR(1), L ocalEventTime TIMESTAMP, < HitColor CHAR(1), LocalEventTime TIMESTAMP,  Age SMALLINT, Sex <calEventTime TIMESTAMP, Age SMALLINT, Sex S MALLINT, Income SMALL < Age SMALLINT, Sex SMALLINT, Income SMALLI NT, Interests INT, <ALLINT, Income SMALLINT, Interests INT,  Robotness SMALLINT, <T, Interests INT, Robotness SMALLINT, R emoteIP BIGINT, Windo <Robotness SMALLINT, RemoteIP BIGINT, Window Name INT, OpenerName <moteIP BIGINT, WindowName INT, OpenerName I NT, HistoryLength SMA <ame INT, OpenerName INT, HistoryLength SMAL LINT, BrowserLanguage <T, HistoryLength SMALLINT, BrowserLanguage CHAR(2), BrowserCount <INT, BrowserLanguage CHAR(2), BrowserCountr y CHAR(2), SocialNetw <HAR(2), BrowserCountry CHAR(2), SocialNetwo rk VARCHAR(128), Soci < CHAR(2), SocialNetwork VARCHAR(128), Socia lAction VARCHAR(128), <k VARCHAR(128), SocialAction VARCHAR(128), HTTPError INT, SendTi <Action VARCHAR(128), HTTPError INT, SendTim ing BIGINT, DNSTiming <TTPError INT, SendTiming BIGINT, DNSTiming BIGINT, ConnectTiming <ng BIGINT, DNSTiming BIGINT, ConnectTiming BIGINT, ResponseStart <IGINT, ConnectTiming BIGINT, ResponseStartT iming BIGINT, Respons <IGINT, ResponseStartTiming BIGINT, Response EndTiming BIGINT, Fet <ming BIGINT, ResponseEndTiming BIGINT, Fetc hTiming BIGINT, Socia <ndTiming BIGINT, FetchTiming BIGINT, Social SourceNetworkID SMALLINT, <Timing BIGINT, SocialSourceNetworkID SMALLINT,  SocialSourcePage VARC <ourceNetworkID SMALLINT, SocialSourcePage VARCH AR(256), ParamPrice B < SocialSourcePage VARCHAR(256), ParamPrice BI GINT, ParamOrderID VA <R(256), ParamPrice BIGINT, ParamOrderID VAR CHAR(80), ParamCurren <INT, ParamOrderID VARCHAR(80), ParamCurrenc y CHAR(3), ParamCurre <HAR(80), ParamCurrency CHAR(3), ParamCurren cyID INT, OpenstatSer < CHAR(3), ParamCurrencyID INT, OpenstatServ iceName VARCHAR(80), <yID INT, OpenstatServiceName VARCHAR(80), O penstatCampaignID VARCHAR <ceName VARCHAR(80), OpenstatCampaignID VARCHAR( 512), OpenstatAdID VA <enstatCampaignID VARCHAR(512), OpenstatAdID VAR CHAR(80), OpenstatSou <12), OpenstatAdID VARCHAR(80), OpenstatSour ceID VARCHAR(256), UT <HAR(80), OpenstatSourceID VARCHAR(256), UTM Source VARCHAR(256), <eID VARCHAR(256), UTMSource VARCHAR(256), U TMMedium VARCHAR(256), <ource VARCHAR(256), UTMMedium VARCHAR(256),  UTMCampaign VARCHAR(407) <MMedium VARCHAR(256), UTMCampaign VARCHAR(407),  UTMContent VARCHAR(2 <UTMCampaign VARCHAR(407), UTMContent VARCHAR(25 6), UTMTerm VARCHAR(4 < UTMContent VARCHAR(256), UTMTerm VARCHAR(43 7), FromTag VARCHAR(4 <), UTMTerm VARCHAR(437), FromTag VARCHAR(42 8), HasGCLID SMALLINT <), FromTag VARCHAR(428), HasGCLID SMALLINT,  RefererHash BIGINT, <), HasGCLID SMALLINT, RefererHash BIGINT,  URLHash BIGINT, CL < RefererHash BIGINT, URLHash BIGINT, CLI D BIGINT, UserIDHash < URLHash BIGINT, CLID BIGINT, UserIDHash BIGINT ); CREATE INDEX h < BIGINT, UserIDHash BIGINT ); CREATE INDEX hi ts_1b_ind ON hits_1b(Cou <IGINT ); CREATE INDEX hits_1b_ind ON hits_1b(Coun terID, EventDate, UserIDH <s_1b_ind ON hits_1b(CounterID, EventDate, UserIDHa sh, EventTime); -CREATE TABLE: name 'hits_1b' already in use -CREATE INDEX: name 'hits_1b_ind' already in use -sql>\q -monetdbd is running -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql> -sql>\q -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>set GLOBAL max_length_for_sort_data = 8388608; -syntax error, unexpected GLOBAL in: "set global" -sql>\q - -times: 1 -query: COPY INTO hits_1b FROM ('/opt/dump/dump_0.3/dump_hits_1b_meshed_utf8.tsv') DELIMITERS '\t'; -spawn mclient -u monetdb -d hits -password: -Welcome to mclient, the MonetDB/SQL interactive terminal (Feb2013-SP3) -Database: MonetDB v11.15.11 (Feb2013-SP3), 'mapi:monetdb://mturlrep13:50000/hits' -Type \q to quit, \? for a list of available commands -auto commit mode: on -sql>COPY INTO hits_1b FROM ('/opt/dump/dump_0.3/dump_hits_1b_meshed_utf8.tsv' <('/opt/dump/dump_0.3/dump_hits_1b_meshed_utf8.tsv')  DELIMITERS '\t'; -\q -stop time: Thu Sep 12 09:25:04 MSK 2013 diff --git a/benchmark/monetdb/queries.sql b/benchmark/monetdb/queries.sql index c17868bb31a..0b9eadb1011 100644 --- a/benchmark/monetdb/queries.sql +++ b/benchmark/monetdb/queries.sql @@ -1,111 +1,43 @@ -SELECT count(*) FROM hits_10m; -SELECT count(*) FROM hits_10m WHERE AdvEngineID <> 0; -SELECT sum(AdvEngineID), count(*), avg(ResolutionWidth) FROM hits_10m; -SELECT sum(UserID) FROM hits_10m; -SELECT count(DISTINCT UserID) FROM hits_10m; -SELECT count(DISTINCT SearchPhrase) FROM hits_10m; -SELECT min(EventDate), max(EventDate) FROM hits_10m; - -SELECT AdvEngineID, count(*) FROM hits_10m WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY count(*) DESC; --- мощная фильтрация. После фильтрации почти ничего не остаётся, но делаем ещё агрегацию.; - -SELECT RegionID, count(DISTINCT UserID) AS u FROM hits_10m GROUP BY RegionID ORDER BY u DESC LIMIT 10; --- агрегация, среднее количество ключей.; - -SELECT RegionID, sum(AdvEngineID), count(*) AS c, avg(ResolutionWidth), count(DISTINCT UserID) FROM hits_10m GROUP BY RegionID ORDER BY count(*) DESC LIMIT 10; --- агрегация, среднее количество ключей, несколько агрегатных функций.; - -SELECT MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; --- мощная фильтрация по строкам, затем агрегация по строкам.; - -SELECT MobilePhone, MobilePhoneModel, count(DISTINCT UserID) AS u FROM hits_10m WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; --- мощная фильтрация по строкам, затем агрегация по паре из числа и строки.; - -SELECT SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; --- средняя фильтрация по строкам, затем агрегация по строкам, большое количество ключей.; - -SELECT SearchPhrase, count(DISTINCT UserID) AS u FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; --- агрегация чуть сложнее.; - -SELECT SearchEngineID, SearchPhrase, count(*) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; --- агрегация по числу и строке, большое количество ключей.; - -SELECT UserID, count(*) FROM hits_10m GROUP BY UserID ORDER BY count(*) DESC LIMIT 10; --- агрегация по очень большому количеству ключей, может не хватить оперативки.; - -SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase ORDER BY count(*) DESC LIMIT 10; --- ещё более сложная агрегация.; - -SELECT UserID, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, SearchPhrase LIMIT 10; --- то же самое, но без сортировки.; - -SELECT UserID, extract (minute from EventTime) AS m, SearchPhrase, count(*) FROM hits_10m GROUP BY UserID, m, SearchPhrase ORDER BY count(*) DESC LIMIT 10; --- ещё более сложная агрегация, не стоит выполнять на больших таблицах.; - -SELECT UserID FROM hits_10m WHERE UserID = 1234567890; --- мощная фильтрация по столбцу типа UInt64.; - -SELECT count(*) FROM hits_10m WHERE URL LIKE '%metrika%'; --- фильтрация по поиску подстроки в строке.; - -SELECT SearchPhrase, MAX(URL), count(*) FROM hits_10m WHERE URL LIKE '%metrika%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; --- вынимаем большие столбцы, фильтрация по строке.; - -SELECT SearchPhrase, MAX(URL), MAX(Title), count(*) AS c, count(DISTINCT UserID) FROM hits_10m WHERE Title LIKE '%Яндекс%' AND URL <> '%.yandex.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY count(*) DESC LIMIT 10; --- чуть больше столбцы.; - -SELECT * FROM hits_10m WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; --- плохой запрос - вынимаем все столбцы.; - -SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; --- большая сортировка.; - -SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; --- большая сортировка по строкам.; - -SELECT SearchPhrase FROM hits_10m WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; --- большая сортировка по кортежу.; - -SELECT CounterID, avg(length(URL)) AS l, count(*) FROM hits_10m WHERE URL <> '' GROUP BY CounterID HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; --- считаем средние длины URL для крупных счётчиков.; - -SELECT SUBSTRING( SUBSTRING(Referer, POSITION('//' IN Referer) + 2), 1, ifthenelse( (0 > POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1), 0, POSITION('/' IN SUBSTRING(Referer, POSITION('//' IN Referer) + 2)) - 1 ) ) AS k, avg(length(Referer)) AS l, count(*) AS c, MAX(Referer) FROM hits_10m WHERE Referer <> '' GROUP BY k HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; --- то же самое, но с разбивкой по доменам.; - -SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits_10m; --- много тупых агрегатных функций.; - -SELECT SearchEngineID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY count(*) DESC LIMIT 10; --- сложная агрегация, для больших таблиц может не хватить оперативки.; - -SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; --- агрегация по двум полям, которая ничего не агрегирует. Для больших таблиц выполнить не получится.; - -SELECT WatchID, ClientIP, count(*) AS c, sum(Refresh), avg(ResolutionWidth) FROM hits_10m GROUP BY WatchID, ClientIP ORDER BY count(*) DESC LIMIT 10; --- то же самое, но ещё и без фильтрации.; - -SELECT URL, count(*) FROM hits_10m GROUP BY URL ORDER BY count(*) DESC LIMIT 10; --- агрегация по URL.; - -SELECT 1, URL, count(*) FROM hits_10m GROUP BY 1, URL ORDER BY count(*) DESC LIMIT 10; --- агрегация по URL и числу.; - -SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, count(*) FROM hits_10m GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY count(*) DESC LIMIT 10; - -SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; - - -SELECT Title, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; - - -SELECT URL, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; - - -SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN SearchEngineID = 0 AND AdvEngineID = 0 THEN Referer ELSE '' END AS Src, URL AS Dst, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; - - -SELECT URLHash, EventDate, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = 6202628419148573758 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100000; - -SELECT WindowClientWidth, WindowClientHeight, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = 6202628419148573758 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; - -SELECT EventTime - extract (SECOND from EventTime) AS M, count(*) AS PageViews FROM hits_10m WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY M ORDER BY M; \ No newline at end of file +SELECT count(*) FROM {table}; +SELECT count(*) FROM {table} WHERE "AdvEngineID" <> 0; +SELECT sum("AdvEngineID"), count(*), avg("ResolutionWidth") FROM {table}; +SELECT sum("UserID") FROM {table}; +SELECT COUNT(DISTINCT "UserID") FROM {table}; +SELECT COUNT(DISTINCT "SearchPhrase") FROM {table}; +SELECT min("EventDate"), max("EventDate") FROM {table}; +SELECT "AdvEngineID", count(*) FROM {table} WHERE "AdvEngineID" <> 0 GROUP BY "AdvEngineID" ORDER BY count(*) DESC; +SELECT "RegionID", COUNT(DISTINCT "UserID") AS u FROM {table} GROUP BY "RegionID" ORDER BY u DESC LIMIT 10; +SELECT "RegionID", sum("AdvEngineID"), count(*) AS c, avg("ResolutionWidth"), COUNT(DISTINCT "UserID") FROM {table} GROUP BY "RegionID" ORDER BY c DESC LIMIT 10; +SELECT "MobilePhoneModel", COUNT(DISTINCT "UserID") AS u FROM {table} WHERE "MobilePhoneModel" <> '' GROUP BY "MobilePhoneModel" ORDER BY u DESC LIMIT 10; +SELECT "MobilePhone", "MobilePhoneModel", COUNT(DISTINCT "UserID") AS u FROM {table} WHERE "MobilePhoneModel" <> '' GROUP BY "MobilePhone", "MobilePhoneModel" ORDER BY u DESC LIMIT 10; +SELECT "SearchPhrase", count(*) AS c FROM {table} WHERE "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; +SELECT "SearchPhrase", COUNT(DISTINCT "UserID") AS u FROM {table} WHERE "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY u DESC LIMIT 10; +SELECT "SearchEngineID", "SearchPhrase", count(*) AS c FROM {table} WHERE "SearchPhrase" <> '' GROUP BY "SearchEngineID", "SearchPhrase" ORDER BY c DESC LIMIT 10; +SELECT "UserID", count(*) FROM {table} GROUP BY "UserID" ORDER BY count(*) DESC LIMIT 10; +SELECT "UserID", "SearchPhrase", count(*) FROM {table} GROUP BY "UserID", "SearchPhrase" ORDER BY count(*) DESC LIMIT 10; +SELECT "UserID", "SearchPhrase", count(*) FROM {table} GROUP BY "UserID", "SearchPhrase" LIMIT 10; +SELECT "UserID", extract(minute FROM "EventTime") AS m, "SearchPhrase", count(*) FROM {table} GROUP BY "UserID", m, "SearchPhrase" ORDER BY count(*) DESC LIMIT 10; +SELECT "UserID" FROM {table} WHERE "UserID" = -6101065172474983726; +SELECT count(*) FROM {table} WHERE "URL" LIKE '%metrika%'; +SELECT "SearchPhrase", min("URL"), count(*) AS c FROM {table} WHERE "URL" LIKE '%metrika%' AND "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; +SELECT "SearchPhrase", min("URL"), min("Title"), count(*) AS c, COUNT(DISTINCT "UserID") FROM {table} WHERE "Title" LIKE '%Яндекс%' AND "URL" NOT LIKE '%.yandex.%' AND "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; +SELECT * FROM {table} WHERE "URL" LIKE '%metrika%' ORDER BY "EventTime" LIMIT 10; +SELECT "SearchPhrase" FROM {table} WHERE "SearchPhrase" <> '' ORDER BY "EventTime" LIMIT 10; +SELECT "SearchPhrase" FROM {table} WHERE "SearchPhrase" <> '' ORDER BY "SearchPhrase" LIMIT 10; +SELECT "SearchPhrase" FROM {table} WHERE "SearchPhrase" <> '' ORDER BY "EventTime", "SearchPhrase" LIMIT 10; +SELECT "CounterID", avg(length("URL")) AS l, count(*) AS c FROM {table} WHERE "URL" <> '' GROUP BY "CounterID" HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT sys.getdomain("Referer") AS key, avg(length("Referer")) AS l, count(*) AS c, min("Referer") FROM {table} WHERE "Referer" <> '' GROUP BY key HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT sum("ResolutionWidth"), sum("ResolutionWidth" + 1), sum("ResolutionWidth" + 2), sum("ResolutionWidth" + 3), sum("ResolutionWidth" + 4), sum("ResolutionWidth" + 5), sum("ResolutionWidth" + 6), sum("ResolutionWidth" + 7), sum("ResolutionWidth" + 8), sum("ResolutionWidth" + 9), sum("ResolutionWidth" + 10), sum("ResolutionWidth" + 11), sum("ResolutionWidth" + 12), sum("ResolutionWidth" + 13), sum("ResolutionWidth" + 14), sum("ResolutionWidth" + 15), sum("ResolutionWidth" + 16), sum("ResolutionWidth" + 17), sum("ResolutionWidth" + 18), sum("ResolutionWidth" + 19), sum("ResolutionWidth" + 20), sum("ResolutionWidth" + 21), sum("ResolutionWidth" + 22), sum("ResolutionWidth" + 23), sum("ResolutionWidth" + 24), sum("ResolutionWidth" + 25), sum("ResolutionWidth" + 26), sum("ResolutionWidth" + 27), sum("ResolutionWidth" + 28), sum("ResolutionWidth" + 29), sum("ResolutionWidth" + 30), sum("ResolutionWidth" + 31), sum("ResolutionWidth" + 32), sum("ResolutionWidth" + 33), sum("ResolutionWidth" + 34), sum("ResolutionWidth" + 35), sum("ResolutionWidth" + 36), sum("ResolutionWidth" + 37), sum("ResolutionWidth" + 38), sum("ResolutionWidth" + 39), sum("ResolutionWidth" + 40), sum("ResolutionWidth" + 41), sum("ResolutionWidth" + 42), sum("ResolutionWidth" + 43), sum("ResolutionWidth" + 44), sum("ResolutionWidth" + 45), sum("ResolutionWidth" + 46), sum("ResolutionWidth" + 47), sum("ResolutionWidth" + 48), sum("ResolutionWidth" + 49), sum("ResolutionWidth" + 50), sum("ResolutionWidth" + 51), sum("ResolutionWidth" + 52), sum("ResolutionWidth" + 53), sum("ResolutionWidth" + 54), sum("ResolutionWidth" + 55), sum("ResolutionWidth" + 56), sum("ResolutionWidth" + 57), sum("ResolutionWidth" + 58), sum("ResolutionWidth" + 59), sum("ResolutionWidth" + 60), sum("ResolutionWidth" + 61), sum("ResolutionWidth" + 62), sum("ResolutionWidth" + 63), sum("ResolutionWidth" + 64), sum("ResolutionWidth" + 65), sum("ResolutionWidth" + 66), sum("ResolutionWidth" + 67), sum("ResolutionWidth" + 68), sum("ResolutionWidth" + 69), sum("ResolutionWidth" + 70), sum("ResolutionWidth" + 71), sum("ResolutionWidth" + 72), sum("ResolutionWidth" + 73), sum("ResolutionWidth" + 74), sum("ResolutionWidth" + 75), sum("ResolutionWidth" + 76), sum("ResolutionWidth" + 77), sum("ResolutionWidth" + 78), sum("ResolutionWidth" + 79), sum("ResolutionWidth" + 80), sum("ResolutionWidth" + 81), sum("ResolutionWidth" + 82), sum("ResolutionWidth" + 83), sum("ResolutionWidth" + 84), sum("ResolutionWidth" + 85), sum("ResolutionWidth" + 86), sum("ResolutionWidth" + 87), sum("ResolutionWidth" + 88), sum("ResolutionWidth" + 89) FROM {table}; +SELECT "SearchEngineID", "ClientIP", count(*) AS c, sum("Refresh"), avg("ResolutionWidth") FROM {table} WHERE "SearchPhrase" <> '' GROUP BY "SearchEngineID", "ClientIP" ORDER BY c DESC LIMIT 10; +SELECT "WatchID", "ClientIP", count(*) AS c, sum("Refresh"), avg("ResolutionWidth") FROM {table} WHERE "SearchPhrase" <> '' GROUP BY "WatchID", "ClientIP" ORDER BY c DESC LIMIT 10; +SELECT "WatchID", "ClientIP", count(*) AS c, sum("Refresh"), avg("ResolutionWidth") FROM {table} GROUP BY "WatchID", "ClientIP" ORDER BY c DESC LIMIT 10; +SELECT "URL", count(*) AS c FROM {table} GROUP BY "URL" ORDER BY c DESC LIMIT 10; +SELECT 1, "URL", count(*) AS c FROM {table} GROUP BY 1, "URL" ORDER BY c DESC LIMIT 10; +SELECT "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3, count(*) AS c FROM {table} GROUP BY "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3 ORDER BY c DESC LIMIT 10; +SELECT "URL", count(*) AS "PageViews" FROM {table} WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "DontCountHits" = 0 AND "Refresh" = 0 AND "URL" <> '' GROUP BY "URL" ORDER BY "PageViews" DESC LIMIT 10; +SELECT "Title", count(*) AS "PageViews" FROM {table} WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "DontCountHits" = 0 AND "Refresh" = 0 AND "Title" <> '' GROUP BY "Title" ORDER BY "PageViews" DESC LIMIT 10; +SELECT "URL", count(*) AS "PageViews" FROM {table} WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 AND "IsLink" <> 0 AND "IsDownload" = 0 GROUP BY "URL" ORDER BY "PageViews" DESC LIMIT 1000; +SELECT "TraficSourceID", "SearchEngineID", "AdvEngineID", CASE WHEN ("SearchEngineID" = 0 AND "AdvEngineID" = 0) THEN "Referer" ELSE '' END AS Src, "URL" AS Dst, count(*) AS "PageViews" FROM {table} WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 GROUP BY "TraficSourceID", "SearchEngineID", "AdvEngineID", CASE WHEN ("SearchEngineID" = 0 AND "AdvEngineID" = 0) THEN "Referer" ELSE '' END, "URL" ORDER BY "PageViews" DESC LIMIT 1000; +SELECT "URLHash", "EventDate", count(*) AS "PageViews" FROM {table} WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 AND "TraficSourceID" IN (-1, 6) AND "RefererHash" = 686716256552154761 GROUP BY "URLHash", "EventDate" ORDER BY "PageViews" DESC LIMIT 100; +SELECT "WindowClientWidth", "WindowClientHeight", count(*) AS "PageViews" FROM {table} WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 AND "DontCountHits" = 0 AND "URLHash" = 686716256552154761 GROUP BY "WindowClientWidth", "WindowClientHeight" ORDER BY "PageViews" DESC LIMIT 10000; +SELECT DATE_TRUNC('minute', "EventTime") AS "Minute", count(*) AS "PageViews" FROM {table} WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-02' AND "Refresh" = 0 AND "DontCountHits" = 0 GROUP BY DATE_TRUNC('minute', "EventTime") ORDER BY DATE_TRUNC('minute', "EventTime"); diff --git a/benchmark/monetdb/expect.tcl b/benchmark/monetdb/send-query old mode 100644 new mode 100755 similarity index 62% rename from benchmark/monetdb/expect.tcl rename to benchmark/monetdb/send-query index 85f5fd2ae4c..6817302e2d5 --- a/benchmark/monetdb/expect.tcl +++ b/benchmark/monetdb/send-query @@ -1,5 +1,4 @@ -#!/usr/bin/env bash -#!/bin/expect +#!/usr/bin/expect # Set timeout set timeout 600 @@ -7,14 +6,14 @@ set timeout 600 # Get arguments set query [lindex $argv 0] -spawn mclient -u monetdb -d hits +spawn mclient -u monetdb -d test --timer=clock expect "password:" send "monetdb\r" expect "sql>" -send "$query\r" +send "$query;\r" expect "sql>" send "\\q\r" -expect eof \ No newline at end of file +expect eof From e4d02f6b512e6ab852682f7f5131060febde579a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed, 12 Aug 2020 06:03:47 +0300 Subject: [PATCH 278/374] Update for useless script #13599 --- utils/check-style/format.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/check-style/format.sh b/utils/check-style/format.sh index 4f8de342c8d..66ab65c483c 100755 --- a/utils/check-style/format.sh +++ b/utils/check-style/format.sh @@ -6,10 +6,10 @@ cd `readlink -f $(dirname $0)`/../.. clang_format=`bash -c "compgen -c clang-format | grep 'clang-format-[[:digit:]]' | sort --version-sort --reverse | head -n1"` if [ ! -z $clang_format ]; then - find base dbms utils -name *.cpp -or -name *.h -exec $clang_format -i {} + ; + find base src programs utils -name *.cpp -or -name *.h -exec $clang_format -i {} + ; else echo clang-format missing. try to install: echo sudo apt install clang-format echo or - echo sudo apt install clang-format-3.9 + echo sudo apt install clang-format-10 fi From 360f4d93efeaf58be7bb1b832358055d05a639a5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed, 12 Aug 2020 06:13:41 +0300 Subject: [PATCH 279/374] Add usability report --- benchmark/monetdb/usability.md | 1165 ++++++++++++++++++++++++++++++++ 1 file changed, 1165 insertions(+) create mode 100644 benchmark/monetdb/usability.md diff --git a/benchmark/monetdb/usability.md b/benchmark/monetdb/usability.md new file mode 100644 index 00000000000..e1328483cb5 --- /dev/null +++ b/benchmark/monetdb/usability.md @@ -0,0 +1,1165 @@ +Go to https://www.monetdb.org/ + +The graphical design of the website is a bit old-fashioned but I do not afraid. + +Dowload now. +Latest binary releases. +Ubuntu & Debian. + +https://www.monetdb.org/downloads/deb/ + +Go to the server where you want to install MonetDB. +``` +$ sudo mcedit /etc/apt/sources.list.d/monetdb.list +``` +Write: +``` +deb https://dev.monetdb.org/downloads/deb/ bionic monetdb +``` + +``` +$ wget --output-document=- https://www.monetdb.org/downloads/MonetDB-GPG-KEY | sudo apt-key add - + +$ sudo apt update +$ sudo apt install monetdb5-sql monetdb-client + +$ sudo systemctl enable monetdbd +$ sudo systemctl start monetdbd +$ sudo usermod -a -G monetdb $USER +``` + +Logout and login back to your server. + +Tutorial: +https://www.monetdb.org/Documentation/UserGuide/Tutorial + +Creating the database: + +``` +$ sudo mkdir /opt/monetdb +$ sudo chmod 777 /opt/monetdb +$ monetdbd create /opt/monetdb + +$ monetdbd start /opt/monetdb +cannot remove socket files +``` + +Don't know what it's doing but I hope it's Ok to ignore. + +``` +$ monetdb create test +created database in maintenance mode: test + +$ monetdb release test +taken database out of maintenance mode: test +``` + +Run client: +``` +$ mclient -u monetdb -d test +``` + +Type password: monetdb + +``` +$ mclient -u monetdb -d test +password: +Welcome to mclient, the MonetDB/SQL interactive terminal (Jun2020-SP1) +Database: MonetDB v11.37.11 (Jun2020-SP1), 'mapi:monetdb://mtlog-perftest03j:50000/test' +FOLLOW US on https://twitter.com/MonetDB or https://github.com/MonetDB/MonetDB +Type \q to quit, \? for a list of available commands +auto commit mode: on +sql>SELECT 1 +more>; ++------+ +| %2 | ++======+ +| 1 | ++------+ +1 tuple +``` + +Yes, it works. +The only downside is the lack of whitespace after `sql>`. + +Upload the dataset. + +``` +CREATE TABLE hits +( + "WatchID" BIGINT, + "JavaEnable" TINYINT, + "Title" TEXT, + "GoodEvent" SMALLINT, + "EventTime" TIMESTAMP, + "EventDate" Date, + "CounterID" INTEGER, + "ClientIP" INTEGER, + "RegionID" INTEGER, + "UserID" BIGINT, + "CounterClass" TINYINT, + "OS" TINYINT, + "UserAgent" TINYINT, + "URL" TEXT, + "Referer" TEXT, + "Refresh" TINYINT, + "RefererCategoryID" SMALLINT, + "RefererRegionID" INTEGER, + "URLCategoryID" SMALLINT, + "URLRegionID" INTEGER, + "ResolutionWidth" SMALLINT, + "ResolutionHeight" SMALLINT, + "ResolutionDepth" TINYINT, + "FlashMajor" TINYINT, + "FlashMinor" TINYINT, + "FlashMinor2" TEXT, + "NetMajor" TINYINT, + "NetMinor" TINYINT, + "UserAgentMajor" SMALLINT, + "UserAgentMinor" TEXT(16), + "CookieEnable" TINYINT, + "JavascriptEnable" TINYINT, + "IsMobile" TINYINT, + "MobilePhone" TINYINT, + "MobilePhoneModel" TEXT, + "Params" TEXT, + "IPNetworkID" INTEGER, + "TraficSourceID" TINYINT, + "SearchEngineID" SMALLINT, + "SearchPhrase" TEXT, + "AdvEngineID" TINYINT, + "IsArtifical" TINYINT, + "WindowClientWidth" SMALLINT, + "WindowClientHeight" SMALLINT, + "ClientTimeZone" SMALLINT, + "ClientEventTime" TIMESTAMP, + "SilverlightVersion1" TINYINT, + "SilverlightVersion2" TINYINT, + "SilverlightVersion3" INTEGER, + "SilverlightVersion4" SMALLINT, + "PageCharset" TEXT, + "CodeVersion" INTEGER, + "IsLink" TINYINT, + "IsDownload" TINYINT, + "IsNotBounce" TINYINT, + "FUniqID" BIGINT, + "OriginalURL" TEXT, + "HID" INTEGER, + "IsOldCounter" TINYINT, + "IsEvent" TINYINT, + "IsParameter" TINYINT, + "DontCountHits" TINYINT, + "WithHash" TINYINT, + "HitColor" TEXT(8), + "LocalEventTime" TIMESTAMP, + "Age" TINYINT, + "Sex" TINYINT, + "Income" TINYINT, + "Interests" SMALLINT, + "Robotness" TINYINT, + "RemoteIP" INTEGER, + "WindowName" INTEGER, + "OpenerName" INTEGER, + "HistoryLength" SMALLINT, + "BrowserLanguage" TEXT(16), + "BrowserCountry" TEXT(16), + "SocialNetwork" TEXT, + "SocialAction" TEXT, + "HTTPError" SMALLINT, + "SendTiming" INTEGER, + "DNSTiming" INTEGER, + "ConnectTiming" INTEGER, + "ResponseStartTiming" INTEGER, + "ResponseEndTiming" INTEGER, + "FetchTiming" INTEGER, + "SocialSourceNetworkID" TINYINT, + "SocialSourcePage" TEXT, + "ParamPrice" BIGINT, + "ParamOrderID" TEXT, + "ParamCurrency" TEXT, + "ParamCurrencyID" SMALLINT, + "OpenstatServiceName" TEXT, + "OpenstatCampaignID" TEXT, + "OpenstatAdID" TEXT, + "OpenstatSourceID" TEXT, + "UTMSource" TEXT, + "UTMMedium" TEXT, + "UTMCampaign" TEXT, + "UTMContent" TEXT, + "UTMTerm" TEXT, + "FromTag" TEXT, + "HasGCLID" TINYINT, + "RefererHash" BIGINT, + "URLHash" BIGINT, + "CLID" INTEGER +); + +operation successful +``` + +``` +sql>SELECT * FROM hits; ++---------+------------+-------+-----------+-----------+-----------+-----------+----------+----------+--------+--------------+----+-----------+-----+---------+---------+-------------------+ +| WatchID | JavaEnable | Title | GoodEvent | EventTime | EventDate | CounterID | ClientIP | RegionID | UserID | CounterClass | OS | UserAgent | URL | Referer | Refresh | RefererCategoryID |> ++=========+============+=======+===========+===========+===========+===========+==========+==========+========+==============+====+===========+=====+=========+=========+===================+ ++---------+------------+-------+-----------+-----------+-----------+-----------+----------+----------+--------+--------------+----+-----------+-----+---------+---------+-------------------+ +0 tuples !88 columns dropped! +note: to disable dropping columns and/or truncating fields use \w-1 +``` + +Perfect. + +https://www.monetdb.org/Documentation/Reference/MonetDBClientApplications/mclient - broken link on page https://www.monetdb.org/Documentation/ServerAdministration/QueryTiming + + +COPY command: https://www.monetdb.org/Documentation/SQLreference/SQLSyntaxOverview#COPY_INTO_FROM + +`COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated.csv' USING DELIMITERS ',', '\n', '"';` + +``` +sql>COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated.csv' USING DELIMITERS ',', '\n', '"'; +Failed to import table 'hits', line 55390 field Robotness 'tinyint' expected in '-128' +``` + +TINYINT - 8 bit signed integer between -127 and 127 +The smallest negative number is not supported by any of the types. + +It makes impossible to store the values of 64bit identifiers in BIGINT. + +Maybe it's a trick to optimize NULLs? + +Let's just cheat and add one to all the most negative numbers while exporting dataset from ClickHouse... + +``` +SELECT + toInt64(WatchID) = -9223372036854775808 ? -9223372036854775807 : toInt64(WatchID), + toInt8(JavaEnable) = -128 ? -127 : toInt8(JavaEnable), + toValidUTF8(toString(Title)), + toInt16(GoodEvent) = -32768 ? -32767 : toInt16(GoodEvent), + EventTime, + EventDate, + toInt32(CounterID) = -2147483648 ? -2147483647 : toInt32(CounterID), + toInt32(ClientIP) = -2147483648 ? -2147483647 : toInt32(ClientIP), + toInt32(RegionID) = -2147483648 ? -2147483647 : toInt32(RegionID), + toInt64(UserID) = -9223372036854775808 ? -9223372036854775807 : toInt64(UserID), + toInt8(CounterClass) = -128 ? -127 : toInt8(CounterClass), + toInt8(OS) = -128 ? -127 : toInt8(OS), + toInt8(UserAgent) = -128 ? -127 : toInt8(UserAgent), + toValidUTF8(toString(URL)), + toValidUTF8(toString(Referer)), + toInt8(Refresh) = -128 ? -127 : toInt8(Refresh), + toInt16(RefererCategoryID) = -32768 ? -32767 : toInt16(RefererCategoryID), + toInt32(RefererRegionID) = -2147483648 ? -2147483647 : toInt32(RefererRegionID), + toInt16(URLCategoryID) = -32768 ? -32767 : toInt16(URLCategoryID), + toInt32(URLRegionID) = -2147483648 ? -2147483647 : toInt32(URLRegionID), + toInt16(ResolutionWidth) = -32768 ? -32767 : toInt16(ResolutionWidth), + toInt16(ResolutionHeight) = -32768 ? -32767 : toInt16(ResolutionHeight), + toInt8(ResolutionDepth) = -128 ? -127 : toInt8(ResolutionDepth), + toInt8(FlashMajor) = -128 ? -127 : toInt8(FlashMajor), + toInt8(FlashMinor) = -128 ? -127 : toInt8(FlashMinor), + toValidUTF8(toString(FlashMinor2)), + toInt8(NetMajor) = -128 ? -127 : toInt8(NetMajor), + toInt8(NetMinor) = -128 ? -127 : toInt8(NetMinor), + toInt16(UserAgentMajor) = -32768 ? -32767 : toInt16(UserAgentMajor), + toValidUTF8(toString(UserAgentMinor)), + toInt8(CookieEnable) = -128 ? -127 : toInt8(CookieEnable), + toInt8(JavascriptEnable) = -128 ? -127 : toInt8(JavascriptEnable), + toInt8(IsMobile) = -128 ? -127 : toInt8(IsMobile), + toInt8(MobilePhone) = -128 ? -127 : toInt8(MobilePhone), + toValidUTF8(toString(MobilePhoneModel)), + toValidUTF8(toString(Params)), + toInt32(IPNetworkID) = -2147483648 ? -2147483647 : toInt32(IPNetworkID), + toInt8(TraficSourceID) = -128 ? -127 : toInt8(TraficSourceID), + toInt16(SearchEngineID) = -32768 ? -32767 : toInt16(SearchEngineID), + toValidUTF8(toString(SearchPhrase)), + toInt8(AdvEngineID) = -128 ? -127 : toInt8(AdvEngineID), + toInt8(IsArtifical) = -128 ? -127 : toInt8(IsArtifical), + toInt16(WindowClientWidth) = -32768 ? -32767 : toInt16(WindowClientWidth), + toInt16(WindowClientHeight) = -32768 ? -32767 : toInt16(WindowClientHeight), + toInt16(ClientTimeZone) = -32768 ? -32767 : toInt16(ClientTimeZone), + ClientEventTime, + toInt8(SilverlightVersion1) = -128 ? -127 : toInt8(SilverlightVersion1), + toInt8(SilverlightVersion2) = -128 ? -127 : toInt8(SilverlightVersion2), + toInt32(SilverlightVersion3) = -2147483648 ? -2147483647 : toInt32(SilverlightVersion3), + toInt16(SilverlightVersion4) = -32768 ? -32767 : toInt16(SilverlightVersion4), + toValidUTF8(toString(PageCharset)), + toInt32(CodeVersion) = -2147483648 ? -2147483647 : toInt32(CodeVersion), + toInt8(IsLink) = -128 ? -127 : toInt8(IsLink), + toInt8(IsDownload) = -128 ? -127 : toInt8(IsDownload), + toInt8(IsNotBounce) = -128 ? -127 : toInt8(IsNotBounce), + toInt64(FUniqID) = -9223372036854775808 ? -9223372036854775807 : toInt64(FUniqID), + toValidUTF8(toString(OriginalURL)), + toInt32(HID) = -2147483648 ? -2147483647 : toInt32(HID), + toInt8(IsOldCounter) = -128 ? -127 : toInt8(IsOldCounter), + toInt8(IsEvent) = -128 ? -127 : toInt8(IsEvent), + toInt8(IsParameter) = -128 ? -127 : toInt8(IsParameter), + toInt8(DontCountHits) = -128 ? -127 : toInt8(DontCountHits), + toInt8(WithHash) = -128 ? -127 : toInt8(WithHash), + toValidUTF8(toString(HitColor)), + LocalEventTime, + toInt8(Age) = -128 ? -127 : toInt8(Age), + toInt8(Sex) = -128 ? -127 : toInt8(Sex), + toInt8(Income) = -128 ? -127 : toInt8(Income), + toInt16(Interests) = -32768 ? -32767 : toInt16(Interests), + toInt8(Robotness) = -128 ? -127 : toInt8(Robotness), + toInt32(RemoteIP) = -2147483648 ? -2147483647 : toInt32(RemoteIP), + toInt32(WindowName) = -2147483648 ? -2147483647 : toInt32(WindowName), + toInt32(OpenerName) = -2147483648 ? -2147483647 : toInt32(OpenerName), + toInt16(HistoryLength) = -32768 ? -32767 : toInt16(HistoryLength), + toValidUTF8(toString(BrowserLanguage)), + toValidUTF8(toString(BrowserCountry)), + toValidUTF8(toString(SocialNetwork)), + toValidUTF8(toString(SocialAction)), + toInt16(HTTPError) = -32768 ? -32767 : toInt16(HTTPError), + toInt32(SendTiming) = -2147483648 ? -2147483647 : toInt32(SendTiming), + toInt32(DNSTiming) = -2147483648 ? -2147483647 : toInt32(DNSTiming), + toInt32(ConnectTiming) = -2147483648 ? -2147483647 : toInt32(ConnectTiming), + toInt32(ResponseStartTiming) = -2147483648 ? -2147483647 : toInt32(ResponseStartTiming), + toInt32(ResponseEndTiming) = -2147483648 ? -2147483647 : toInt32(ResponseEndTiming), + toInt32(FetchTiming) = -2147483648 ? -2147483647 : toInt32(FetchTiming), + toInt8(SocialSourceNetworkID) = -128 ? -127 : toInt8(SocialSourceNetworkID), + toValidUTF8(toString(SocialSourcePage)), + toInt64(ParamPrice) = -9223372036854775808 ? -9223372036854775807 : toInt64(ParamPrice), + toValidUTF8(toString(ParamOrderID)), + toValidUTF8(toString(ParamCurrency)), + toInt16(ParamCurrencyID) = -32768 ? -32767 : toInt16(ParamCurrencyID), + toValidUTF8(toString(OpenstatServiceName)), + toValidUTF8(toString(OpenstatCampaignID)), + toValidUTF8(toString(OpenstatAdID)), + toValidUTF8(toString(OpenstatSourceID)), + toValidUTF8(toString(UTMSource)), + toValidUTF8(toString(UTMMedium)), + toValidUTF8(toString(UTMCampaign)), + toValidUTF8(toString(UTMContent)), + toValidUTF8(toString(UTMTerm)), + toValidUTF8(toString(FromTag)), + toInt8(HasGCLID) = -128 ? -127 : toInt8(HasGCLID), + toInt64(RefererHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(RefererHash), + toInt64(URLHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(URLHash), + toInt32(CLID) = -2147483648 ? -2147483647 : toInt32(CLID) +FROM hits_100m_obfuscated +INTO OUTFILE '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' +FORMAT CSV; +``` + +Try №2. + +`COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' USING DELIMITERS ',', '\n', '"';` + + +``` +sql>COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' USING DELIMITERS ',', '\n', '"'; +Failed to import table 'hits', line 1: column 106: Leftover data '1526320043,139,7122783580357023164,1,44,2,"http://smeshariki.ru/a-albumshowtopic/8940180","http://video.yandex.ru/yandex.ru/site=&airbag=&srt=0&fu=0",0,0,20,0,22,1917,879,37,15,13,"800",0,0,10,"sO",1,1,0,0,"","",3626245,-1,0,"",0,0,746,459,135,"2013-07-21 15:14:16",0,0,0,0,"windows",1,0,0,0,8675577400349020325,"",1034597214,0,0,0,0,0,"5","2013-07-21 11:14:27",31,1,2,3557,5,1782490839,-1,-1,-1,"S0","� + " +``` + +Looks like it does not support newlines inside string literals. + +Let's dig into https://www.monetdb.org/Documentation/ServerAdministration/LoadingBulkData/CSVBulkLoads + +First, it's better to specify the number of records: +`COPY 100000000 RECORDS INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' USING DELIMITERS ',', '\n', '"';` + +> Quote characters in quoted fields may be escaped with a backslash. + +Ok, then it's TSV, not CSV. Let's create TSV dump... + + +``` +SELECT + toInt64(WatchID) = -9223372036854775808 ? -9223372036854775807 : toInt64(WatchID), + toInt8(JavaEnable) = -128 ? -127 : toInt8(JavaEnable), + toValidUTF8(toString(Title)), + toInt16(GoodEvent) = -32768 ? -32767 : toInt16(GoodEvent), + EventTime, + EventDate, + toInt32(CounterID) = -2147483648 ? -2147483647 : toInt32(CounterID), + toInt32(ClientIP) = -2147483648 ? -2147483647 : toInt32(ClientIP), + toInt32(RegionID) = -2147483648 ? -2147483647 : toInt32(RegionID), + toInt64(UserID) = -9223372036854775808 ? -9223372036854775807 : toInt64(UserID), + toInt8(CounterClass) = -128 ? -127 : toInt8(CounterClass), + toInt8(OS) = -128 ? -127 : toInt8(OS), + toInt8(UserAgent) = -128 ? -127 : toInt8(UserAgent), + toValidUTF8(toString(URL)), + toValidUTF8(toString(Referer)), + toInt8(Refresh) = -128 ? -127 : toInt8(Refresh), + toInt16(RefererCategoryID) = -32768 ? -32767 : toInt16(RefererCategoryID), + toInt32(RefererRegionID) = -2147483648 ? -2147483647 : toInt32(RefererRegionID), + toInt16(URLCategoryID) = -32768 ? -32767 : toInt16(URLCategoryID), + toInt32(URLRegionID) = -2147483648 ? -2147483647 : toInt32(URLRegionID), + toInt16(ResolutionWidth) = -32768 ? -32767 : toInt16(ResolutionWidth), + toInt16(ResolutionHeight) = -32768 ? -32767 : toInt16(ResolutionHeight), + toInt8(ResolutionDepth) = -128 ? -127 : toInt8(ResolutionDepth), + toInt8(FlashMajor) = -128 ? -127 : toInt8(FlashMajor), + toInt8(FlashMinor) = -128 ? -127 : toInt8(FlashMinor), + toValidUTF8(toString(FlashMinor2)), + toInt8(NetMajor) = -128 ? -127 : toInt8(NetMajor), + toInt8(NetMinor) = -128 ? -127 : toInt8(NetMinor), + toInt16(UserAgentMajor) = -32768 ? -32767 : toInt16(UserAgentMajor), + toValidUTF8(toString(UserAgentMinor)), + toInt8(CookieEnable) = -128 ? -127 : toInt8(CookieEnable), + toInt8(JavascriptEnable) = -128 ? -127 : toInt8(JavascriptEnable), + toInt8(IsMobile) = -128 ? -127 : toInt8(IsMobile), + toInt8(MobilePhone) = -128 ? -127 : toInt8(MobilePhone), + toValidUTF8(toString(MobilePhoneModel)), + toValidUTF8(toString(Params)), + toInt32(IPNetworkID) = -2147483648 ? -2147483647 : toInt32(IPNetworkID), + toInt8(TraficSourceID) = -128 ? -127 : toInt8(TraficSourceID), + toInt16(SearchEngineID) = -32768 ? -32767 : toInt16(SearchEngineID), + toValidUTF8(toString(SearchPhrase)), + toInt8(AdvEngineID) = -128 ? -127 : toInt8(AdvEngineID), + toInt8(IsArtifical) = -128 ? -127 : toInt8(IsArtifical), + toInt16(WindowClientWidth) = -32768 ? -32767 : toInt16(WindowClientWidth), + toInt16(WindowClientHeight) = -32768 ? -32767 : toInt16(WindowClientHeight), + toInt16(ClientTimeZone) = -32768 ? -32767 : toInt16(ClientTimeZone), + ClientEventTime, + toInt8(SilverlightVersion1) = -128 ? -127 : toInt8(SilverlightVersion1), + toInt8(SilverlightVersion2) = -128 ? -127 : toInt8(SilverlightVersion2), + toInt32(SilverlightVersion3) = -2147483648 ? -2147483647 : toInt32(SilverlightVersion3), + toInt16(SilverlightVersion4) = -32768 ? -32767 : toInt16(SilverlightVersion4), + toValidUTF8(toString(PageCharset)), + toInt32(CodeVersion) = -2147483648 ? -2147483647 : toInt32(CodeVersion), + toInt8(IsLink) = -128 ? -127 : toInt8(IsLink), + toInt8(IsDownload) = -128 ? -127 : toInt8(IsDownload), + toInt8(IsNotBounce) = -128 ? -127 : toInt8(IsNotBounce), + toInt64(FUniqID) = -9223372036854775808 ? -9223372036854775807 : toInt64(FUniqID), + toValidUTF8(toString(OriginalURL)), + toInt32(HID) = -2147483648 ? -2147483647 : toInt32(HID), + toInt8(IsOldCounter) = -128 ? -127 : toInt8(IsOldCounter), + toInt8(IsEvent) = -128 ? -127 : toInt8(IsEvent), + toInt8(IsParameter) = -128 ? -127 : toInt8(IsParameter), + toInt8(DontCountHits) = -128 ? -127 : toInt8(DontCountHits), + toInt8(WithHash) = -128 ? -127 : toInt8(WithHash), + toValidUTF8(toString(HitColor)), + LocalEventTime, + toInt8(Age) = -128 ? -127 : toInt8(Age), + toInt8(Sex) = -128 ? -127 : toInt8(Sex), + toInt8(Income) = -128 ? -127 : toInt8(Income), + toInt16(Interests) = -32768 ? -32767 : toInt16(Interests), + toInt8(Robotness) = -128 ? -127 : toInt8(Robotness), + toInt32(RemoteIP) = -2147483648 ? -2147483647 : toInt32(RemoteIP), + toInt32(WindowName) = -2147483648 ? -2147483647 : toInt32(WindowName), + toInt32(OpenerName) = -2147483648 ? -2147483647 : toInt32(OpenerName), + toInt16(HistoryLength) = -32768 ? -32767 : toInt16(HistoryLength), + toValidUTF8(toString(BrowserLanguage)), + toValidUTF8(toString(BrowserCountry)), + toValidUTF8(toString(SocialNetwork)), + toValidUTF8(toString(SocialAction)), + toInt16(HTTPError) = -32768 ? -32767 : toInt16(HTTPError), + toInt32(SendTiming) = -2147483648 ? -2147483647 : toInt32(SendTiming), + toInt32(DNSTiming) = -2147483648 ? -2147483647 : toInt32(DNSTiming), + toInt32(ConnectTiming) = -2147483648 ? -2147483647 : toInt32(ConnectTiming), + toInt32(ResponseStartTiming) = -2147483648 ? -2147483647 : toInt32(ResponseStartTiming), + toInt32(ResponseEndTiming) = -2147483648 ? -2147483647 : toInt32(ResponseEndTiming), + toInt32(FetchTiming) = -2147483648 ? -2147483647 : toInt32(FetchTiming), + toInt8(SocialSourceNetworkID) = -128 ? -127 : toInt8(SocialSourceNetworkID), + toValidUTF8(toString(SocialSourcePage)), + toInt64(ParamPrice) = -9223372036854775808 ? -9223372036854775807 : toInt64(ParamPrice), + toValidUTF8(toString(ParamOrderID)), + toValidUTF8(toString(ParamCurrency)), + toInt16(ParamCurrencyID) = -32768 ? -32767 : toInt16(ParamCurrencyID), + toValidUTF8(toString(OpenstatServiceName)), + toValidUTF8(toString(OpenstatCampaignID)), + toValidUTF8(toString(OpenstatAdID)), + toValidUTF8(toString(OpenstatSourceID)), + toValidUTF8(toString(UTMSource)), + toValidUTF8(toString(UTMMedium)), + toValidUTF8(toString(UTMCampaign)), + toValidUTF8(toString(UTMContent)), + toValidUTF8(toString(UTMTerm)), + toValidUTF8(toString(FromTag)), + toInt8(HasGCLID) = -128 ? -127 : toInt8(HasGCLID), + toInt64(RefererHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(RefererHash), + toInt64(URLHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(URLHash), + toInt32(CLID) = -2147483648 ? -2147483647 : toInt32(CLID) +FROM hits_100m_obfuscated +INTO OUTFILE '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' +FORMAT TSV; +``` + +MonetDB client lacks history. + +`mclient -u monetdb -d test --timer=clock` + +`COPY 100000000 RECORDS INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' USING DELIMITERS '\t', '\n', '';` + +``` +sql>COPY 100000000 RECORDS INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' USING DELIMITERS '\t', '\n', ''; +Failed to import table 'hits', +clk: 1.200 sec +``` + +Now it gives incomprehensible error... +Looks like it because of 100000000 RECORDS. +Let's try without it. + +`COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' USING DELIMITERS '\t', '\n', '';` + +Ok, it appeared to work... + +`top -d0.5` + +`mserver5` consumes about 1 CPU core but with strange pauses. + +``` +sql>COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' USING DELIMITERS '\t', '\n', ''; +Failed to import table 'hits', +clk: 2:31 min +``` + +It does not work and there is no explanation available. + +When I type Ctrl+D in CLI, it does not output a line feed into terminal. + +Let's google it. +https://www.monetdb.org/pipermail/users-list/2013-November/007014.html + +Probably it because of no quoting for strings. +Let's create a dump with `|` as a separator, `"` as string quote and C-style escaping. +But it's impossible to create dump in that format in ClickHouse. + +Let's consider using binary format... + +Ok, before we consider binary format, maybe we need to write character literals as E'\t' instead of '\t'? + +`mclient` does not have an option to specify password in command line, it's annoying. +PS. I found how to solve it here: https://www.monetdb.org/Documentation/ServerAdministration/ServerSetupAndConfiguration + +``` +COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' USING DELIMITERS E'\t', E'\n', E''; +``` + +It does not work either: + +``` +sql>COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' USING DELIMITERS E'\t', E'\n', E''; +Failed to import table 'hits', Failed to extend the BAT, perhaps disk full +clk: 1:17 min +``` + +Let's try binary import. But it would not work: + +> For variable length strings, the file must have one C-based string value per line, terminated by a newline, and it is processed without escape character conversion. Fixed length strings are handled the same way. MonetDB assumes that all files are aligned, i.e. the i-th value in each file corresponds to the i-th record in the table. + +According to the docs, there is no way to import strings with line feed characters. + +BTW, the favicon of the MonetDB website makes an impression that the web page is constantly loading (it looks like a spinner). + +Let's cheat again and replace all line feeds in strings to whitespaces and all double quotes to single quotes. + +``` +SELECT + toInt64(WatchID) = -9223372036854775808 ? -9223372036854775807 : toInt64(WatchID), + toInt8(JavaEnable) = -128 ? -127 : toInt8(JavaEnable), + replaceAll(replaceAll(toValidUTF8(toString(Title)), '\n', ' '), '"', '\''), + toInt16(GoodEvent) = -32768 ? -32767 : toInt16(GoodEvent), + EventTime, + EventDate, + toInt32(CounterID) = -2147483648 ? -2147483647 : toInt32(CounterID), + toInt32(ClientIP) = -2147483648 ? -2147483647 : toInt32(ClientIP), + toInt32(RegionID) = -2147483648 ? -2147483647 : toInt32(RegionID), + toInt64(UserID) = -9223372036854775808 ? -9223372036854775807 : toInt64(UserID), + toInt8(CounterClass) = -128 ? -127 : toInt8(CounterClass), + toInt8(OS) = -128 ? -127 : toInt8(OS), + toInt8(UserAgent) = -128 ? -127 : toInt8(UserAgent), + replaceAll(replaceAll(toValidUTF8(toString(URL)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(Referer)), '\n', ' '), '"', '\''), + toInt8(Refresh) = -128 ? -127 : toInt8(Refresh), + toInt16(RefererCategoryID) = -32768 ? -32767 : toInt16(RefererCategoryID), + toInt32(RefererRegionID) = -2147483648 ? -2147483647 : toInt32(RefererRegionID), + toInt16(URLCategoryID) = -32768 ? -32767 : toInt16(URLCategoryID), + toInt32(URLRegionID) = -2147483648 ? -2147483647 : toInt32(URLRegionID), + toInt16(ResolutionWidth) = -32768 ? -32767 : toInt16(ResolutionWidth), + toInt16(ResolutionHeight) = -32768 ? -32767 : toInt16(ResolutionHeight), + toInt8(ResolutionDepth) = -128 ? -127 : toInt8(ResolutionDepth), + toInt8(FlashMajor) = -128 ? -127 : toInt8(FlashMajor), + toInt8(FlashMinor) = -128 ? -127 : toInt8(FlashMinor), + replaceAll(replaceAll(toValidUTF8(toString(FlashMinor2)), '\n', ' '), '"', '\''), + toInt8(NetMajor) = -128 ? -127 : toInt8(NetMajor), + toInt8(NetMinor) = -128 ? -127 : toInt8(NetMinor), + toInt16(UserAgentMajor) = -32768 ? -32767 : toInt16(UserAgentMajor), + replaceAll(replaceAll(toValidUTF8(toString(UserAgentMinor)), '\n', ' '), '"', '\''), + toInt8(CookieEnable) = -128 ? -127 : toInt8(CookieEnable), + toInt8(JavascriptEnable) = -128 ? -127 : toInt8(JavascriptEnable), + toInt8(IsMobile) = -128 ? -127 : toInt8(IsMobile), + toInt8(MobilePhone) = -128 ? -127 : toInt8(MobilePhone), + replaceAll(replaceAll(toValidUTF8(toString(MobilePhoneModel)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(Params)), '\n', ' '), '"', '\''), + toInt32(IPNetworkID) = -2147483648 ? -2147483647 : toInt32(IPNetworkID), + toInt8(TraficSourceID) = -128 ? -127 : toInt8(TraficSourceID), + toInt16(SearchEngineID) = -32768 ? -32767 : toInt16(SearchEngineID), + replaceAll(replaceAll(toValidUTF8(toString(SearchPhrase)), '\n', ' '), '"', '\''), + toInt8(AdvEngineID) = -128 ? -127 : toInt8(AdvEngineID), + toInt8(IsArtifical) = -128 ? -127 : toInt8(IsArtifical), + toInt16(WindowClientWidth) = -32768 ? -32767 : toInt16(WindowClientWidth), + toInt16(WindowClientHeight) = -32768 ? -32767 : toInt16(WindowClientHeight), + toInt16(ClientTimeZone) = -32768 ? -32767 : toInt16(ClientTimeZone), + ClientEventTime, + toInt8(SilverlightVersion1) = -128 ? -127 : toInt8(SilverlightVersion1), + toInt8(SilverlightVersion2) = -128 ? -127 : toInt8(SilverlightVersion2), + toInt32(SilverlightVersion3) = -2147483648 ? -2147483647 : toInt32(SilverlightVersion3), + toInt16(SilverlightVersion4) = -32768 ? -32767 : toInt16(SilverlightVersion4), + replaceAll(replaceAll(toValidUTF8(toString(PageCharset)), '\n', ' '), '"', '\''), + toInt32(CodeVersion) = -2147483648 ? -2147483647 : toInt32(CodeVersion), + toInt8(IsLink) = -128 ? -127 : toInt8(IsLink), + toInt8(IsDownload) = -128 ? -127 : toInt8(IsDownload), + toInt8(IsNotBounce) = -128 ? -127 : toInt8(IsNotBounce), + toInt64(FUniqID) = -9223372036854775808 ? -9223372036854775807 : toInt64(FUniqID), + replaceAll(replaceAll(toValidUTF8(toString(OriginalURL)), '\n', ' '), '"', '\''), + toInt32(HID) = -2147483648 ? -2147483647 : toInt32(HID), + toInt8(IsOldCounter) = -128 ? -127 : toInt8(IsOldCounter), + toInt8(IsEvent) = -128 ? -127 : toInt8(IsEvent), + toInt8(IsParameter) = -128 ? -127 : toInt8(IsParameter), + toInt8(DontCountHits) = -128 ? -127 : toInt8(DontCountHits), + toInt8(WithHash) = -128 ? -127 : toInt8(WithHash), + replaceAll(replaceAll(toValidUTF8(toString(HitColor)), '\n', ' '), '"', '\''), + LocalEventTime, + toInt8(Age) = -128 ? -127 : toInt8(Age), + toInt8(Sex) = -128 ? -127 : toInt8(Sex), + toInt8(Income) = -128 ? -127 : toInt8(Income), + toInt16(Interests) = -32768 ? -32767 : toInt16(Interests), + toInt8(Robotness) = -128 ? -127 : toInt8(Robotness), + toInt32(RemoteIP) = -2147483648 ? -2147483647 : toInt32(RemoteIP), + toInt32(WindowName) = -2147483648 ? -2147483647 : toInt32(WindowName), + toInt32(OpenerName) = -2147483648 ? -2147483647 : toInt32(OpenerName), + toInt16(HistoryLength) = -32768 ? -32767 : toInt16(HistoryLength), + replaceAll(replaceAll(toValidUTF8(toString(BrowserLanguage)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(BrowserCountry)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(SocialNetwork)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(SocialAction)), '\n', ' '), '"', '\''), + toInt16(HTTPError) = -32768 ? -32767 : toInt16(HTTPError), + toInt32(SendTiming) = -2147483648 ? -2147483647 : toInt32(SendTiming), + toInt32(DNSTiming) = -2147483648 ? -2147483647 : toInt32(DNSTiming), + toInt32(ConnectTiming) = -2147483648 ? -2147483647 : toInt32(ConnectTiming), + toInt32(ResponseStartTiming) = -2147483648 ? -2147483647 : toInt32(ResponseStartTiming), + toInt32(ResponseEndTiming) = -2147483648 ? -2147483647 : toInt32(ResponseEndTiming), + toInt32(FetchTiming) = -2147483648 ? -2147483647 : toInt32(FetchTiming), + toInt8(SocialSourceNetworkID) = -128 ? -127 : toInt8(SocialSourceNetworkID), + replaceAll(replaceAll(toValidUTF8(toString(SocialSourcePage)), '\n', ' '), '"', '\''), + toInt64(ParamPrice) = -9223372036854775808 ? -9223372036854775807 : toInt64(ParamPrice), + replaceAll(replaceAll(toValidUTF8(toString(ParamOrderID)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(ParamCurrency)), '\n', ' '), '"', '\''), + toInt16(ParamCurrencyID) = -32768 ? -32767 : toInt16(ParamCurrencyID), + replaceAll(replaceAll(toValidUTF8(toString(OpenstatServiceName)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(OpenstatCampaignID)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(OpenstatAdID)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(OpenstatSourceID)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(UTMSource)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(UTMMedium)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(UTMCampaign)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(UTMContent)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(UTMTerm)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(FromTag)), '\n', ' '), '"', '\''), + toInt8(HasGCLID) = -128 ? -127 : toInt8(HasGCLID), + toInt64(RefererHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(RefererHash), + toInt64(URLHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(URLHash), + toInt32(CLID) = -2147483648 ? -2147483647 : toInt32(CLID) +FROM hits_100m_obfuscated +INTO OUTFILE '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' +FORMAT CSV; +``` + +Another try: + +``` +COPY 100000000 RECORDS INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' USING DELIMITERS ',', '\n', '"'; +``` + +Does not work. + +``` +Failed to import table 'hits', +clk: 1.091 sec +``` + +Another try: + +``` +COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' USING DELIMITERS ',', '\n', '"'; +``` + +Does not work. + +``` +Failed to import table 'hits', line 79128: record too long +clk: 1.194 sec +``` + +Ok, the error message becomes more meaningful. Looks like MonetDB does not support long TEXT. +Let's continue reading docs... + +> CLOB | TEXT | STRING | CHARACTER LARGE OBJECT: UTF-8 character string with unbounded length + +It must be unbounded! +But maybe there is global limit on record length... + +https://www.monetdb.org/search/node?keys=record+length +https://www.monetdb.org/search/node?keys=record+too+long + +The docs search did not give an answer. Let's search in the internet... + +https://www.monetdb.org/pipermail/users-list/2017-August/009930.html + +It's unclear what is the record numbering scheme - from 1 or from 0. +But when I took at the records with + +``` +head -n79128 hits_100m_obfuscated_monetdb.csv | tail -n1 +head -n79129 hits_100m_obfuscated_monetdb.csv | tail -n1 +``` + +they don't look too long. + +Ok, let's try to load data with "best effort" mode that MonetDB offers. + +``` +COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' USING DELIMITERS ',', '\n', '"' BEST EFFORT; +``` + +But it loaded just 79127 rows. That's not what I need. + +``` +79127 affected rows +clk: 1.684 sec +``` + +The TRUNCATE query works: + +``` +TRUNCATE TABLE hits; +``` + +Let's check if the record 79127 is really any longer than other records. + +Let's remove all length like `TEXT(16)` from CREATE TABLE statement... + +``` +DROP TABLE hits; + +CREATE TABLE hits +( + "WatchID" BIGINT, + "JavaEnable" TINYINT, + "Title" TEXT, + "GoodEvent" SMALLINT, + "EventTime" TIMESTAMP, + "EventDate" Date, + "CounterID" INTEGER, + "ClientIP" INTEGER, + "RegionID" INTEGER, + "UserID" BIGINT, + "CounterClass" TINYINT, + "OS" TINYINT, + "UserAgent" TINYINT, + "URL" TEXT, + "Referer" TEXT, + "Refresh" TINYINT, + "RefererCategoryID" SMALLINT, + "RefererRegionID" INTEGER, + "URLCategoryID" SMALLINT, + "URLRegionID" INTEGER, + "ResolutionWidth" SMALLINT, + "ResolutionHeight" SMALLINT, + "ResolutionDepth" TINYINT, + "FlashMajor" TINYINT, + "FlashMinor" TINYINT, + "FlashMinor2" TEXT, + "NetMajor" TINYINT, + "NetMinor" TINYINT, + "UserAgentMajor" SMALLINT, + "UserAgentMinor" TEXT, + "CookieEnable" TINYINT, + "JavascriptEnable" TINYINT, + "IsMobile" TINYINT, + "MobilePhone" TINYINT, + "MobilePhoneModel" TEXT, + "Params" TEXT, + "IPNetworkID" INTEGER, + "TraficSourceID" TINYINT, + "SearchEngineID" SMALLINT, + "SearchPhrase" TEXT, + "AdvEngineID" TINYINT, + "IsArtifical" TINYINT, + "WindowClientWidth" SMALLINT, + "WindowClientHeight" SMALLINT, + "ClientTimeZone" SMALLINT, + "ClientEventTime" TIMESTAMP, + "SilverlightVersion1" TINYINT, + "SilverlightVersion2" TINYINT, + "SilverlightVersion3" INTEGER, + "SilverlightVersion4" SMALLINT, + "PageCharset" TEXT, + "CodeVersion" INTEGER, + "IsLink" TINYINT, + "IsDownload" TINYINT, + "IsNotBounce" TINYINT, + "FUniqID" BIGINT, + "OriginalURL" TEXT, + "HID" INTEGER, + "IsOldCounter" TINYINT, + "IsEvent" TINYINT, + "IsParameter" TINYINT, + "DontCountHits" TINYINT, + "WithHash" TINYINT, + "HitColor" TEXT, + "LocalEventTime" TIMESTAMP, + "Age" TINYINT, + "Sex" TINYINT, + "Income" TINYINT, + "Interests" SMALLINT, + "Robotness" TINYINT, + "RemoteIP" INTEGER, + "WindowName" INTEGER, + "OpenerName" INTEGER, + "HistoryLength" SMALLINT, + "BrowserLanguage" TEXT, + "BrowserCountry" TEXT, + "SocialNetwork" TEXT, + "SocialAction" TEXT, + "HTTPError" SMALLINT, + "SendTiming" INTEGER, + "DNSTiming" INTEGER, + "ConnectTiming" INTEGER, + "ResponseStartTiming" INTEGER, + "ResponseEndTiming" INTEGER, + "FetchTiming" INTEGER, + "SocialSourceNetworkID" TINYINT, + "SocialSourcePage" TEXT, + "ParamPrice" BIGINT, + "ParamOrderID" TEXT, + "ParamCurrency" TEXT, + "ParamCurrencyID" SMALLINT, + "OpenstatServiceName" TEXT, + "OpenstatCampaignID" TEXT, + "OpenstatAdID" TEXT, + "OpenstatSourceID" TEXT, + "UTMSource" TEXT, + "UTMMedium" TEXT, + "UTMCampaign" TEXT, + "UTMContent" TEXT, + "UTMTerm" TEXT, + "FromTag" TEXT, + "HasGCLID" TINYINT, + "RefererHash" BIGINT, + "URLHash" BIGINT, + "CLID" INTEGER +); +``` + +``` +COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' USING DELIMITERS ',', '\n', '"'; +``` + +Unfortunately it did not help. + +``` +Failed to import table 'hits', line 79128: record too long +clk: 1.224 sec +``` + +Let's check actual record lengths: + +``` +$ cat hits_100m_obfuscated_monetdb.csv | awk 'BEGIN { FS = "\n"; max_length = 0 } { ++num; l = length($1); if (l > max_length) { max_length = l; print l, "in line", num } }' +588 in line 1 +705 in line 2 +786 in line 4 +788 in line 5 +913 in line 9 +917 in line 38 +996 in line 56 +1007 in line 113 +1008 in line 115 +1015 in line 183 +1147 in line 207 +1180 in line 654 +1190 in line 656 +1191 in line 795 +1446 in line 856 +1519 in line 1572 +1646 in line 1686 +1700 in line 3084 +1701 in line 3086 +2346 in line 4013 +2630 in line 8245 +3035 in line 8248 +3257 in line 8289 +3762 in line 8307 +5536 in line 8376 +5568 in line 71721 +6507 in line 92993 +6734 in line 163169 +7706 in line 473542 +8368 in line 2803973 +9375 in line 5433559 +``` + +No, there is nothing special in line 79128. + +Let's try to load just a single line into MonetDB to figure out what is so special about this line. + +``` +head -n79128 hits_100m_obfuscated_monetdb.csv | tail -n1 > hits_100m_obfuscated_monetdb.csv1 +``` + +``` +COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv1' USING DELIMITERS ',', '\n', '"'; +``` + +`Failed to import table 'hits', line 1: incomplete record at end of file` + +Now we have another error. +Ok. I understand that MonetDB is just parsing CSV with C-style escaping rules as TSV. + +I will try to stick with TSV. + +``` +COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' USING DELIMITERS '\t'; +``` + +Nothing good happened, it failed after 2.5 minutes with incomprehensible error: + +``` +Failed to import table 'hits', +clk: 2:30 min +``` + +Let's replace all backslashes from CSV. + +``` +SELECT + toInt64(WatchID) = -9223372036854775808 ? -9223372036854775807 : toInt64(WatchID), + toInt8(JavaEnable) = -128 ? -127 : toInt8(JavaEnable), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(Title)), '\n', ' '), '"', '\''), '\\', '/'), + toInt16(GoodEvent) = -32768 ? -32767 : toInt16(GoodEvent), + EventTime, + EventDate, + toInt32(CounterID) = -2147483648 ? -2147483647 : toInt32(CounterID), + toInt32(ClientIP) = -2147483648 ? -2147483647 : toInt32(ClientIP), + toInt32(RegionID) = -2147483648 ? -2147483647 : toInt32(RegionID), + toInt64(UserID) = -9223372036854775808 ? -9223372036854775807 : toInt64(UserID), + toInt8(CounterClass) = -128 ? -127 : toInt8(CounterClass), + toInt8(OS) = -128 ? -127 : toInt8(OS), + toInt8(UserAgent) = -128 ? -127 : toInt8(UserAgent), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(URL)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(Referer)), '\n', ' '), '"', '\''), '\\', '/'), + toInt8(Refresh) = -128 ? -127 : toInt8(Refresh), + toInt16(RefererCategoryID) = -32768 ? -32767 : toInt16(RefererCategoryID), + toInt32(RefererRegionID) = -2147483648 ? -2147483647 : toInt32(RefererRegionID), + toInt16(URLCategoryID) = -32768 ? -32767 : toInt16(URLCategoryID), + toInt32(URLRegionID) = -2147483648 ? -2147483647 : toInt32(URLRegionID), + toInt16(ResolutionWidth) = -32768 ? -32767 : toInt16(ResolutionWidth), + toInt16(ResolutionHeight) = -32768 ? -32767 : toInt16(ResolutionHeight), + toInt8(ResolutionDepth) = -128 ? -127 : toInt8(ResolutionDepth), + toInt8(FlashMajor) = -128 ? -127 : toInt8(FlashMajor), + toInt8(FlashMinor) = -128 ? -127 : toInt8(FlashMinor), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(FlashMinor2)), '\n', ' '), '"', '\''), '\\', '/'), + toInt8(NetMajor) = -128 ? -127 : toInt8(NetMajor), + toInt8(NetMinor) = -128 ? -127 : toInt8(NetMinor), + toInt16(UserAgentMajor) = -32768 ? -32767 : toInt16(UserAgentMajor), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UserAgentMinor)), '\n', ' '), '"', '\''), '\\', '/'), + toInt8(CookieEnable) = -128 ? -127 : toInt8(CookieEnable), + toInt8(JavascriptEnable) = -128 ? -127 : toInt8(JavascriptEnable), + toInt8(IsMobile) = -128 ? -127 : toInt8(IsMobile), + toInt8(MobilePhone) = -128 ? -127 : toInt8(MobilePhone), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(MobilePhoneModel)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(Params)), '\n', ' '), '"', '\''), '\\', '/'), + toInt32(IPNetworkID) = -2147483648 ? -2147483647 : toInt32(IPNetworkID), + toInt8(TraficSourceID) = -128 ? -127 : toInt8(TraficSourceID), + toInt16(SearchEngineID) = -32768 ? -32767 : toInt16(SearchEngineID), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(SearchPhrase)), '\n', ' '), '"', '\''), '\\', '/'), + toInt8(AdvEngineID) = -128 ? -127 : toInt8(AdvEngineID), + toInt8(IsArtifical) = -128 ? -127 : toInt8(IsArtifical), + toInt16(WindowClientWidth) = -32768 ? -32767 : toInt16(WindowClientWidth), + toInt16(WindowClientHeight) = -32768 ? -32767 : toInt16(WindowClientHeight), + toInt16(ClientTimeZone) = -32768 ? -32767 : toInt16(ClientTimeZone), + ClientEventTime, + toInt8(SilverlightVersion1) = -128 ? -127 : toInt8(SilverlightVersion1), + toInt8(SilverlightVersion2) = -128 ? -127 : toInt8(SilverlightVersion2), + toInt32(SilverlightVersion3) = -2147483648 ? -2147483647 : toInt32(SilverlightVersion3), + toInt16(SilverlightVersion4) = -32768 ? -32767 : toInt16(SilverlightVersion4), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(PageCharset)), '\n', ' '), '"', '\''), '\\', '/'), + toInt32(CodeVersion) = -2147483648 ? -2147483647 : toInt32(CodeVersion), + toInt8(IsLink) = -128 ? -127 : toInt8(IsLink), + toInt8(IsDownload) = -128 ? -127 : toInt8(IsDownload), + toInt8(IsNotBounce) = -128 ? -127 : toInt8(IsNotBounce), + toInt64(FUniqID) = -9223372036854775808 ? -9223372036854775807 : toInt64(FUniqID), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OriginalURL)), '\n', ' '), '"', '\''), '\\', '/'), + toInt32(HID) = -2147483648 ? -2147483647 : toInt32(HID), + toInt8(IsOldCounter) = -128 ? -127 : toInt8(IsOldCounter), + toInt8(IsEvent) = -128 ? -127 : toInt8(IsEvent), + toInt8(IsParameter) = -128 ? -127 : toInt8(IsParameter), + toInt8(DontCountHits) = -128 ? -127 : toInt8(DontCountHits), + toInt8(WithHash) = -128 ? -127 : toInt8(WithHash), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(HitColor)), '\n', ' '), '"', '\''), '\\', '/'), + LocalEventTime, + toInt8(Age) = -128 ? -127 : toInt8(Age), + toInt8(Sex) = -128 ? -127 : toInt8(Sex), + toInt8(Income) = -128 ? -127 : toInt8(Income), + toInt16(Interests) = -32768 ? -32767 : toInt16(Interests), + toInt8(Robotness) = -128 ? -127 : toInt8(Robotness), + toInt32(RemoteIP) = -2147483648 ? -2147483647 : toInt32(RemoteIP), + toInt32(WindowName) = -2147483648 ? -2147483647 : toInt32(WindowName), + toInt32(OpenerName) = -2147483648 ? -2147483647 : toInt32(OpenerName), + toInt16(HistoryLength) = -32768 ? -32767 : toInt16(HistoryLength), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(BrowserLanguage)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(BrowserCountry)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(SocialNetwork)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(SocialAction)), '\n', ' '), '"', '\''), '\\', '/'), + toInt16(HTTPError) = -32768 ? -32767 : toInt16(HTTPError), + toInt32(SendTiming) = -2147483648 ? -2147483647 : toInt32(SendTiming), + toInt32(DNSTiming) = -2147483648 ? -2147483647 : toInt32(DNSTiming), + toInt32(ConnectTiming) = -2147483648 ? -2147483647 : toInt32(ConnectTiming), + toInt32(ResponseStartTiming) = -2147483648 ? -2147483647 : toInt32(ResponseStartTiming), + toInt32(ResponseEndTiming) = -2147483648 ? -2147483647 : toInt32(ResponseEndTiming), + toInt32(FetchTiming) = -2147483648 ? -2147483647 : toInt32(FetchTiming), + toInt8(SocialSourceNetworkID) = -128 ? -127 : toInt8(SocialSourceNetworkID), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(SocialSourcePage)), '\n', ' '), '"', '\''), '\\', '/'), + toInt64(ParamPrice) = -9223372036854775808 ? -9223372036854775807 : toInt64(ParamPrice), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(ParamOrderID)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(ParamCurrency)), '\n', ' '), '"', '\''), '\\', '/'), + toInt16(ParamCurrencyID) = -32768 ? -32767 : toInt16(ParamCurrencyID), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OpenstatServiceName)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OpenstatCampaignID)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OpenstatAdID)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OpenstatSourceID)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMSource)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMMedium)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMCampaign)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMContent)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMTerm)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(FromTag)), '\n', ' '), '"', '\''), '\\', '/'), + toInt8(HasGCLID) = -128 ? -127 : toInt8(HasGCLID), + toInt64(RefererHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(RefererHash), + toInt64(URLHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(URLHash), + toInt32(CLID) = -2147483648 ? -2147483647 : toInt32(CLID) +FROM hits_100m_obfuscated +INTO OUTFILE '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' +FORMAT CSV; +``` + +Another try: +``` +COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' USING DELIMITERS ',', '\n', '"'; +``` + +MonetDB still takes about one CPU core to load the data, while docs promised me parallel load. +And there are strange pauses... + +``` +sql>COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' USING DELIMITERS ',', '\n', '"'; +Failed to import table 'hits', +clk: 2:14 min +``` + +It still does not work!!! + +Let's look at the logs. +Logs are found in + +``` +/var/log/monetdb$ sudo less merovingian.log +``` + +And the log is the following: +``` +2020-08-12 03:44:03 ERR test[542123]: #wrkr0-hits: GDKextendf: !ERROR: could not extend file: No space left on device +2020-08-12 03:44:03 ERR test[542123]: #wrkr0-hits: MT_mremap: !ERROR: MT_mremap(/var/monetdb5/dbfarm/test/bat/10/1013.theap,0x7f0b2c3b0000,344981504,349175808): GDKextendf() failed +2020-08-12 03:44:03 ERR test[542123]: #wrkr0-hits: GDKmremap: !ERROR: requesting virtual memory failed; memory requested: 349175808, memory in use: 113744056, virtual memory in use: 3124271288 +2020-08-12 03:44:03 ERR test[542123]: #wrkr0-hits: HEAPextend: !ERROR: failed to extend to 349175808 for 10/1013.theap: GDKmremap() failed +2020-08-12 03:44:04 ERR test[542123]: #client14: createExceptionInternal: !ERROR: SQLException:importTable:42000!Failed to import table 'hits', +``` + +So, why it was created my "db farm" inside /var/monetdb5/ instead of /opt/ as I requested? + +Let's stop MonetDB and symlink /var/monetdb5 to /opt + +``` +COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' USING DELIMITERS ',', E'\n', '"'; +``` + +It started to load data... but after ten minutes it looks like stopped processing it, but the query does not finish. + +There is no `SHOW PROCESSLIST` command. + +I see the following message in `merovingian.log`: +``` +2020-08-12 04:03:53 ERR test[682554]: #prod-hits: createExceptionInternal: !ERROR: MALException:sql.copy_from:line 40694471: record too long (EOS found) +``` + +What does EOS mean? It should not be "end of stream" because we have 100 000 000 records, that's more than just 40 694 471. + +Another try with TSV: + +``` +COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' USING DELIMITERS '\t'; +``` + +Ok, it's doing something at least for ten minues... +Ok, it's doing something at least for twenty minues... + +``` +100000000 affected rows +clk: 28:02 min +``` + +Finally it has loaded data successfuly in 28 minutes. It's not fast - just below 60 000 rows per second. + +But the second query from the test does not work: + +``` +sql>SELECT count(*) FROM hits WHERE AdvEngineID <> 0; +SELECT: identifier 'advengineid' unknown +clk: 0.328 ms +sql>DESC TABLE hits +more>; +syntax error, unexpected DESC in: "desc" +clk: 0.471 ms +sql>DESCRIBE TABLE hits; +syntax error, unexpected IDENT in: "describe" +clk: 0.245 ms +sql>SHOW CREATE TABLE hits; +syntax error, unexpected IDENT in: "show" +clk: 0.246 ms +sql>\d hits; +table sys.hits; does not exist +sql>\d test.hits; +table test.hits; does not exist +sql>\d +TABLE sys.hits +sql>\t +Current time formatter: clock +sql>\dd +unknown sub-command for \d: d +sql>help +more>; +syntax error, unexpected IDENT in: "help" +clk: 0.494 ms +sql>SELECT count(*) FROM hits; ++-----------+ +| %1 | ++===========+ +| 100000001 | ++-----------+ +1 tuple +clk: 1.949 ms +sql>SELECT * FROM hits LIMIT 1; +``` + +And the query `SELECT * FROM hits LIMIT 1` does not finish in reasonable time. +It took 3:23 min. + +Ok, I has to put all identifiers in quotes in my queries, like this: + +``` +SELECT count(*) FROM hits WHERE "AdvEngineID" <> 0; +``` + +There is no approximate count distinct functions. Will use exact count distinct instead. + +Run queries: +`./benchmark.sh` + +It works rather slowly. It is barely using more than a single CPU core. And there is nothing about performance tuning in: +https://www.monetdb.org/Documentation/ServerAdministration/ServerSetupAndConfiguration + From c8d3959c4fdd6701f239abce2145ecbbd331eee1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed, 12 Aug 2020 06:31:10 +0300 Subject: [PATCH 280/374] Add drop caches --- benchmark/monetdb/benchmark.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/benchmark/monetdb/benchmark.sh b/benchmark/monetdb/benchmark.sh index ad98bcbe8f3..3766f1a8eef 100755 --- a/benchmark/monetdb/benchmark.sh +++ b/benchmark/monetdb/benchmark.sh @@ -1,6 +1,7 @@ #!/bin/bash grep -v -P '^#' queries.sql | sed -e 's/{table}/hits/' | while read query; do + echo 3 | sudo tee /proc/sys/vm/drop_caches echo "$query"; for i in {1..3}; do From b8e6de7ca3f387ba9a6f7e64f4f79b2c247d02c3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed, 12 Aug 2020 06:31:42 +0300 Subject: [PATCH 281/374] Create index --- benchmark/monetdb/usability.md | 37 ++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/benchmark/monetdb/usability.md b/benchmark/monetdb/usability.md index e1328483cb5..7f544485af2 100644 --- a/benchmark/monetdb/usability.md +++ b/benchmark/monetdb/usability.md @@ -1163,3 +1163,40 @@ Run queries: It works rather slowly. It is barely using more than a single CPU core. And there is nothing about performance tuning in: https://www.monetdb.org/Documentation/ServerAdministration/ServerSetupAndConfiguration + +The last 7 queries from the benchmark benefit from index. Let's create it: + +`CREATE INDEX hits_idx ON hits ("CounterID", "EventDate");` + +``` +sql>CREATE INDEX hits_idx ON hits ("CounterID", "EventDate"); +operation successful +clk: 5.374 sec +``` + +Ok. It was created quickly and successful. +Let's check how does it speed up queries... + +``` +sql>SELECT DATE_TRUNC('minute', "EventTime") AS "Minute", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-02' AND "Refresh" = 0 AND "DontCountHits" = 0 GROUP BY DATE_TRUNC('minute', "EventTime") ORDER BY DATE_TRUNC('minute', "EventTime"); ++--------+-----------+ +| Minute | PageViews | ++========+===========+ ++--------+-----------+ +0 tuples +clk: 4.042 sec +``` + +There is almost no difference. +And the trivial index lookup query is still slow: + +``` +sql>SELECT count(*) FROM hits WHERE "CounterID" = 62; ++--------+ +| %1 | ++========+ +| 738172 | ++--------+ +1 tuple +clk: 1.406 sec +``` From 2acfe2324b3d3b79e931767f4f07fd04c75a38be Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed, 12 Aug 2020 06:44:49 +0300 Subject: [PATCH 282/374] Preliminary results --- benchmark/monetdb/log.txt | 295 ++++++++++++++++++ benchmark/monetdb/usability.md | 6 + .../benchmark/dbms/results/005_monetdb.json | 145 +++------ 3 files changed, 346 insertions(+), 100 deletions(-) create mode 100644 benchmark/monetdb/log.txt diff --git a/benchmark/monetdb/log.txt b/benchmark/monetdb/log.txt new file mode 100644 index 00000000000..920556019bd --- /dev/null +++ b/benchmark/monetdb/log.txt @@ -0,0 +1,295 @@ +SELECT count(*) FROM hits; +1 tuple +clk: 1.250 ms +1 tuple +clk: 1.215 ms +1 tuple +clk: 1.187 ms +SELECT count(*) FROM hits WHERE "AdvEngineID" <> 0; +1 tuple +clk: 1.468 sec +1 tuple +clk: 1.726 sec +1 tuple +clk: 2.369 sec +SELECT sum("AdvEngineID"), count(*), avg("ResolutionWidth") FROM hits; +1 tuple +clk: 748.860 ms +1 tuple +clk: 655.206 ms +1 tuple +clk: 686.095 ms +SELECT sum("UserID") FROM hits; +clk: 683.854 ms +clk: 702.824 ms +clk: 697.182 ms +SELECT COUNT(DISTINCT "UserID") FROM hits; +1 tuple +clk: 8.918 sec +1 tuple +clk: 11.265 sec +1 tuple +clk: 6.662 sec +SELECT COUNT(DISTINCT "SearchPhrase") FROM hits; +1 tuple +clk: 15.144 sec +1 tuple +clk: 15.054 sec +1 tuple +clk: 15.074 sec +SELECT min("EventDate"), max("EventDate") FROM hits; +1 tuple +clk: 695.901 ms +1 tuple +clk: 671.020 ms +1 tuple +clk: 669.029 ms +SELECT "AdvEngineID", count(*) FROM hits WHERE "AdvEngineID" <> 0 GROUP BY "AdvEngineID" ORDER BY count(*) DESC; +18 tuples +clk: 1.425 sec +18 tuples +clk: 1.421 sec +18 tuples +clk: 1.441 sec +SELECT "RegionID", COUNT(DISTINCT "UserID") AS u FROM hits GROUP BY "RegionID" ORDER BY u DESC LIMIT 10; +10 tuples +clk: 10.135 sec +10 tuples +clk: 10.278 sec +10 tuples +clk: 9.905 sec +SELECT "RegionID", sum("AdvEngineID"), count(*) AS c, avg("ResolutionWidth"), COUNT(DISTINCT "UserID") FROM hits GROUP BY "RegionID" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 8.765 sec +10 tuples +clk: 8.729 sec +10 tuples +clk: 8.504 sec +SELECT "MobilePhoneModel", COUNT(DISTINCT "UserID") AS u FROM hits WHERE "MobilePhoneModel" <> '' GROUP BY "MobilePhoneModel" ORDER BY u DESC LIMIT 10; +10 tuples +clk: 3.196 sec +10 tuples +clk: 3.347 sec +10 tuples +clk: 3.276 sec +SELECT "MobilePhone", "MobilePhoneModel", COUNT(DISTINCT "UserID") AS u FROM hits WHERE "MobilePhoneModel" <> '' GROUP BY "MobilePhone", "MobilePhoneModel" ORDER BY u DESC LIMIT 10; +10 tuples +clk: 3.360 sec +10 tuples +clk: 3.398 sec +10 tuples +clk: 3.422 sec +SELECT "SearchPhrase", count(*) AS c FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 10.501 sec +10 tuples +clk: 10.388 sec +10 tuples +clk: 10.305 sec +SELECT "SearchPhrase", COUNT(DISTINCT "UserID") AS u FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY u DESC LIMIT 10; +10 tuples +clk: 15.811 sec +10 tuples +clk: 15.463 sec +10 tuples +clk: 15.017 sec +SELECT "SearchEngineID", "SearchPhrase", count(*) AS c FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchEngineID", "SearchPhrase" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 11.336 sec +10 tuples +clk: 11.229 sec +10 tuples +clk: 11.069 sec +SELECT "UserID", count(*) FROM hits GROUP BY "UserID" ORDER BY count(*) DESC LIMIT 10; +10 tuples +clk: 6.965 sec +10 tuples +clk: 6.884 sec +10 tuples +clk: 6.907 sec +SELECT "UserID", "SearchPhrase", count(*) FROM hits GROUP BY "UserID", "SearchPhrase" ORDER BY count(*) DESC LIMIT 10; +10 tuples +clk: 28.985 sec +10 tuples +clk: 29.714 sec +10 tuples +clk: 28.438 sec +SELECT "UserID", "SearchPhrase", count(*) FROM hits GROUP BY "UserID", "SearchPhrase" LIMIT 10; +10 tuples +clk: 27.820 sec +10 tuples +clk: 29.318 sec +10 tuples +clk: 29.515 sec +SELECT "UserID", extract(minute FROM "EventTime") AS m, "SearchPhrase", count(*) FROM hits GROUP BY "UserID", m, "SearchPhrase" ORDER BY count(*) DESC LIMIT 10; +10 tuples +clk: 46.359 sec +10 tuples +clk: 46.382 sec +10 tuples +clk: 46.353 sec +SELECT "UserID" FROM hits WHERE "UserID" = -6101065172474983726; +0 tuples +clk: 750.670 ms +0 tuples +clk: 768.232 ms +0 tuples +clk: 765.361 ms +SELECT count(*) FROM hits WHERE "URL" LIKE '%metrika%'; +1 tuple +clk: 3.938 sec +1 tuple +clk: 3.932 sec +1 tuple +clk: 3.915 sec +SELECT "SearchPhrase", min("URL"), count(*) AS c FROM hits WHERE "URL" LIKE '%metrika%' AND "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 3.742 sec +10 tuples +clk: 3.784 sec +10 tuples +clk: 3.727 sec +SELECT "SearchPhrase", min("URL"), min("Title"), count(*) AS c, COUNT(DISTINCT "UserID") FROM hits WHERE "Title" LIKE '%Яндекс%' AND "URL" NOT LIKE '%.yandex.%' AND "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 6.024 sec +10 tuples +clk: 5.948 sec +10 tuples +clk: 6.001 sec +SELECT * FROM hits WHERE "URL" LIKE '%metrika%' ORDER BY "EventTime" LIMIT 10; +10 tuples !87 columns dropped, 29 fields truncated! +clk: 4.130 sec +10 tuples !87 columns dropped, 29 fields truncated! +clk: 4.138 sec +10 tuples !87 columns dropped, 29 fields truncated! +clk: 4.270 sec +SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "EventTime" LIMIT 10; +10 tuples +clk: 6.659 sec +10 tuples +clk: 6.751 sec +10 tuples +clk: 6.674 sec +SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "SearchPhrase" LIMIT 10; +10 tuples +clk: 6.817 sec +10 tuples +clk: 6.745 sec +10 tuples +clk: 6.793 sec +SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "EventTime", "SearchPhrase" LIMIT 10; +10 tuples +clk: 6.742 sec +10 tuples +clk: 6.673 sec +10 tuples +clk: 6.743 sec +SELECT "CounterID", avg(length("URL")) AS l, count(*) AS c FROM hits WHERE "URL" <> '' GROUP BY "CounterID" HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; +25 tuples +clk: 57.559 sec +25 tuples +clk: 56.546 sec +25 tuples +clk: 58.124 sec +SELECT sys.getdomain("Referer") AS key, avg(length("Referer")) AS l, count(*) AS c, min("Referer") FROM hits WHERE "Referer" <> '' GROUP BY key HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; +clk: 1:00 min +clk: 59.189 sec +clk: 1:00 min +SELECT sum("ResolutionWidth"), sum("ResolutionWidth" + 1), sum("ResolutionWidth" + 2), sum("ResolutionWidth" + 3), sum("ResolutionWidth" + 4), sum("ResolutionWidth" + 5), sum("ResolutionWidth" + 6), sum("ResolutionWidth" + 7), sum("ResolutionWidth" + 8), sum("ResolutionWidth" + 9), sum("ResolutionWidth" + 10), sum("ResolutionWidth" + 11), sum("ResolutionWidth" + 12), sum("ResolutionWidth" + 13), sum("ResolutionWidth" + 14), sum("ResolutionWidth" + 15), sum("ResolutionWidth" + 16), sum("ResolutionWidth" + 17), sum("ResolutionWidth" + 18), sum("ResolutionWidth" + 19), sum("ResolutionWidth" + 20), sum("ResolutionWidth" + 21), sum("ResolutionWidth" + 22), sum("ResolutionWidth" + 23), sum("ResolutionWidth" + 24), sum("ResolutionWidth" + 25), sum("ResolutionWidth" + 26), sum("ResolutionWidth" + 27), sum("ResolutionWidth" + 28), sum("ResolutionWidth" + 29), sum("ResolutionWidth" + 30), sum("ResolutionWidth" + 31), sum("ResolutionWidth" + 32), sum("ResolutionWidth" + 33), sum("ResolutionWidth" + 34), sum("ResolutionWidth" + 35), sum("ResolutionWidth" + 36), sum("ResolutionWidth" + 37), sum("ResolutionWidth" + 38), sum("ResolutionWidth" + 39), sum("ResolutionWidth" + 40), sum("ResolutionWidth" + 41), sum("ResolutionWidth" + 42), sum("ResolutionWidth" + 43), sum("ResolutionWidth" + 44), sum("ResolutionWidth" + 45), sum("ResolutionWidth" + 46), sum("ResolutionWidth" + 47), sum("ResolutionWidth" + 48), sum("ResolutionWidth" + 49), sum("ResolutionWidth" + 50), sum("ResolutionWidth" + 51), sum("ResolutionWidth" + 52), sum("ResolutionWidth" + 53), sum("ResolutionWidth" + 54), sum("ResolutionWidth" + 55), sum("ResolutionWidth" + 56), sum("ResolutionWidth" + 57), sum("ResolutionWidth" + 58), sum("ResolutionWidth" + 59), sum("ResolutionWidth" + 60), sum("ResolutionWidth" + 61), sum("ResolutionWidth" + 62), sum("ResolutionWidth" + 63), sum("ResolutionWidth" + 64), sum("ResolutionWidth" + 65), sum("ResolutionWidth" + 66), sum("ResolutionWidth" + 67), sum("ResolutionWidth" + 68), sum("ResolutionWidth" + 69), sum("ResolutionWidth" + 70), sum("ResolutionWidth" + 71), sum("ResolutionWidth" + 72), sum("ResolutionWidth" + 73), sum("ResolutionWidth" + 74), sum("ResolutionWidth" + 75), sum("ResolutionWidth" + 76), sum("ResolutionWidth" + 77), sum("ResolutionWidth" + 78), sum("ResolutionWidth" + 79), sum("ResolutionWidth" + 80), sum("ResolutionWidth" + 81), sum("ResolutionWidth" + 82), sum("ResolutionWidth" + 83), sum("ResolutionWidth" + 84), sum("ResolutionWidth" + 85), sum("ResolutionWidth" + 86), sum("ResolutionWidth" + 87), sum("ResolutionWidth" + 88), sum("ResolutionWidth" + 89) FROM hits; +1 tuple !77 columns dropped! +clk: 6.374 sec +1 tuple !77 columns dropped! +clk: 6.428 sec +1 tuple !77 columns dropped! +clk: 6.115 sec +SELECT "SearchEngineID", "ClientIP", count(*) AS c, sum("Refresh"), avg("ResolutionWidth") FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchEngineID", "ClientIP" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 5.336 sec +10 tuples +clk: 5.555 sec +10 tuples +clk: 5.342 sec +SELECT "WatchID", "ClientIP", count(*) AS c, sum("Refresh"), avg("ResolutionWidth") FROM hits WHERE "SearchPhrase" <> '' GROUP BY "WatchID", "ClientIP" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 6.184 sec +10 tuples +clk: 6.252 sec +10 tuples +clk: 6.541 sec +SELECT "WatchID", "ClientIP", count(*) AS c, sum("Refresh"), avg("ResolutionWidth") FROM hits GROUP BY "WatchID", "ClientIP" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 43.900 sec +10 tuples +clk: 42.386 sec +10 tuples +clk: 43.674 sec +SELECT "URL", count(*) AS c FROM hits GROUP BY "URL" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 1:25 min +10 tuples +clk: 1:23 min +10 tuples +clk: 1:23 min +SELECT 1, "URL", count(*) AS c FROM hits GROUP BY 1, "URL" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 1:23 min +10 tuples +clk: 1:22 min +10 tuples +clk: 1:23 min +SELECT "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3, count(*) AS c FROM hits GROUP BY "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3 ORDER BY c DESC LIMIT 10; +10 tuples +clk: 26.289 sec +10 tuples +clk: 25.788 sec +10 tuples +clk: 24.693 sec +SELECT "URL", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "DontCountHits" = 0 AND "Refresh" = 0 AND "URL" <> '' GROUP BY "URL" ORDER BY "PageViews" DESC LIMIT 10; +10 tuples +clk: 4.506 sec +10 tuples +clk: 4.503 sec +10 tuples +clk: 4.496 sec +SELECT "Title", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "DontCountHits" = 0 AND "Refresh" = 0 AND "Title" <> '' GROUP BY "Title" ORDER BY "PageViews" DESC LIMIT 10; +10 tuples +clk: 4.306 sec +10 tuples +clk: 4.337 sec +10 tuples +clk: 4.355 sec +SELECT "URL", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 AND "IsLink" <> 0 AND "IsDownload" = 0 GROUP BY "URL" ORDER BY "PageViews" DESC LIMIT 1000; +1000 tuples +clk: 4.274 sec +1000 tuples +clk: 4.145 sec +1000 tuples +clk: 4.305 sec +SELECT "TraficSourceID", "SearchEngineID", "AdvEngineID", CASE WHEN ("SearchEngineID" = 0 AND "AdvEngineID" = 0) THEN "Referer" ELSE '' END AS Src, "URL" AS Dst, count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 GROUP BY "TraficSourceID", "SearchEngineID", "AdvEngineID", CASE WHEN ("SearchEngineID" = 0 AND "AdvEngineID" = 0) THEN "Referer" ELSE '' END, "URL" ORDER BY "PageViews" DESC LIMIT 1000; +1000 tuples +clk: 3.175 sec +1000 tuples +clk: 3.121 sec +1000 tuples +clk: 3.186 sec +SELECT "URLHash", "EventDate", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 AND "TraficSourceID" IN (-1, 6) AND "RefererHash" = 686716256552154761 GROUP BY "URLHash", "EventDate" ORDER BY "PageViews" DESC LIMIT 100; +0 tuples +clk: 2.600 sec +0 tuples +clk: 2.598 sec +0 tuples +clk: 2.507 sec +SELECT "WindowClientWidth", "WindowClientHeight", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 AND "DontCountHits" = 0 AND "URLHash" = 686716256552154761 GROUP BY "WindowClientWidth", "WindowClientHeight" ORDER BY "PageViews" DESC LIMIT 10000; +0 tuples +clk: 4.239 sec +0 tuples +clk: 4.205 sec +0 tuples +clk: 4.166 sec +SELECT DATE_TRUNC('minute', "EventTime") AS "Minute", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-02' AND "Refresh" = 0 AND "DontCountHits" = 0 GROUP BY DATE_TRUNC('minute', "EventTime") ORDER BY DATE_TRUNC('minute', "EventTime"); +0 tuples +clk: 4.109 sec +0 tuples +clk: 4.222 sec +0 tuples +clk: 4.209 sec diff --git a/benchmark/monetdb/usability.md b/benchmark/monetdb/usability.md index 7f544485af2..c1f35b6bd4e 100644 --- a/benchmark/monetdb/usability.md +++ b/benchmark/monetdb/usability.md @@ -1200,3 +1200,9 @@ sql>SELECT count(*) FROM hits WHERE "CounterID" = 62; 1 tuple clk: 1.406 sec ``` + +How to prepare the benchmark report: + +`grep clk log.txt | awk '{ if ($3 == "ms") { print $2 / 1000; } else if ($3 == "sec") { print $2 } else { print } }'` + + diff --git a/website/benchmark/dbms/results/005_monetdb.json b/website/benchmark/dbms/results/005_monetdb.json index caa335babc9..a29186f0922 100644 --- a/website/benchmark/dbms/results/005_monetdb.json +++ b/website/benchmark/dbms/results/005_monetdb.json @@ -1,110 +1,55 @@ [ { "system": "MonetDB", - "version": "", - "data_size": 10000000, - "time": "", - "comments": "", - "result": - [ - [0.003851, 0.003389, 0.003633], - [0.400058, 0.045117, 0.007999], - [0.207544, 0.022219, 0.033007], - [null, null, null], - [0.873236, 0.61427, 0.564001], - [3.1, 2.8, 2.9], - [0.191616, 0.004704, 0.008579], - [0.056658, 0.049444, 0.035463], - [5.8, 5.8, 6.4], - [5.8, 8.4, 8.5], - [0.458164, 0.319166, 0.413914], - [0.344021, 0.314183, 0.348057], - [6.1, 13.3, 6.1], - [5.8, 5.6, 5.5], - [54.9, 70.0, 58.4], - [0.886465, 0.711899, 0.7329], - [5.8, 5.9, 5.9], - [5.8, 5.9, 6.0], - [20.2, 19.7, 21.0], - [0.331309, 0.000618, 0.085817], - [2.0, 0.056549, 0.09292], - [0.053756, 0.050622, 0.054916], - [2.2, 0.069379, 0.066628], - [13.0, 0.511351, 0.484708], - [0.137787, 0.030937, 0.030501], - [0.039574, 0.027741, 0.025684], - [0.046094, 0.043038, 0.039139], - [4.1, 2.7, 2.7], - [null, null, null], - [1.1, 0.940847, 0.921118], - [17.7, 18.2, 17.5], - [2.0, 0.236466, 0.236223], - [1.3, 1.3, 1.3], - [138.0, 140.0, 173.0], - [null, null, null], - [null, null, null], - [169.0, 175.0, null], - [261.0, 257.0, 268.0], - [3.1, 0.513628, 0.464017], - [262.0, 252.0, 257.0], - [0.524173, 0.066989, 0.123604], - [0.305343, 0.030736, 0.048725], - [2.3, 1.9, 1.9] - ] - }, - - { - "system": "MonetDB", - "version": "", + "version": "11.37.11 (Jun2020-SP1)", "data_size": 100000000, - "time": "", + "time": "2020-08-12", "comments": "", "result": [ - [0.028886, 0.003581, 0.003829], - [2.7, 0.139637, 0.060264], - [1.1, 0.146556, 0.146063], - [null, null, null], - [7.5, 5.6, 5.4], - [15.5, 12.7, 11.5], - [1.0, 0.038566, 0.036837], - [0.687604, 0.026271, 0.030261], - [195.0, 214.0, null], - [254.0, 267.0, 265.0], - [22.9, 20.0, 20.8], - [20.4, 20.8, 23.3], - [97.0, 115.0, null], - [111.0, 84.0, 58.8], - [656.0, null, null], - [6.8, 6.9, 6.7], - [264.0, 234.0, 208.0], - [222.0, 279.0, null], - [1173.0, null, null], - [3.9, 0.000787, 0.032566], - [27.4, 1.7, 2.1], - [4.0, 2.6, 2.6], - [null, null, null], - [209.0, 155, 143], - [10.5, 5.7, 40.9], - [4.8, 4.7, 4.7], - [85.0, 4.7, 5.7], - [52.4, 30.3, 29.1], - [null, null, null], - [null, null, null], - [12.7, 11.2, 11.4], - [392.0, 370.0, null], - [3.2, 1.8, 1.8], - [41.9, 37.0, 40.7], - [null, null, null], - [null, null, null], - [null, null, null], - [6.4, 2.3, 2.1], - [3.6, 1.0, 1.1], - [1.5, 1.2, 1.3], - [7.2, 4.6, 4.9], - [2.8, 0.050916, 0.04148], - [3.0, 0.045935, 0.047277], - [2.5, 0.049119, 0.04828] + [0.00125, 0.001215, 0.001187], + [1.468, 1.726, 2.369], + [0.74886, 0.655206, 0.686095], + [0.683854, 0.702824, 0.697182], + [8.918, 11.265, 6.662], + [15.144, 15.054, 15.074], + [0.695901, 0.67102, 0.669029], + [1.425, 1.421, 1.441], + [10.135, 10.278, 9.905], + [8.765, 8.729, 8.504], + [3.196, 3.347, 3.276], + [3.360, 3.398, 3.422], + [10.501, 10.388, 10.305], + [15.811, 15.463, 15.017], + [11.336, 11.229, 11.069], + [6.965, 6.884, 6.907], + [28.985, 29.714, 28.438], + [27.820, 29.318, 29.515], + [46.359, 46.382, 46.353], + [0.75067, 0.768232, 0.765361], + [3.938, 3.932, 3.915], + [3.742, 3.784, 3.727], + [6.024, 5.948, 6.001], + [4.130, 4.138, 4.270], + [6.659, 6.751, 6.674], + [6.817, 6.745, 6.793], + [6.742, 6.673, 6.743], + [57.559, 56.546, 58.124], + [60, 59.189, 60], + [6.374, 6.428, 6.115], + [5.336, 5.555, 5.342], + [6.184, 6.252, 6.541], + [43.900, 42.386, 43.674], + [85, 83, 83], + [83, 82, 83], + [26.289, 25.788, 24.693], + [4.506, 4.503, 4.496], + [4.306, 4.337, 4.355], + [4.274, 4.145, 4.305], + [3.175, 3.121, 3.186], + [2.600, 2.598, 2.507], + [4.239, 4.205, 4.166], + [4.109, 4.222, 4.209] ] } ] From 7d7a2d0ddef17115fd935405248d0c8d3fc8615f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed, 12 Aug 2020 07:00:04 +0300 Subject: [PATCH 283/374] Added instruction to reproduce the benchmark --- benchmark/monetdb/instruction.md | 356 +++++++++++++++++++++++++++++++ benchmark/monetdb/usability.md | 8 +- 2 files changed, 363 insertions(+), 1 deletion(-) create mode 100644 benchmark/monetdb/instruction.md diff --git a/benchmark/monetdb/instruction.md b/benchmark/monetdb/instruction.md new file mode 100644 index 00000000000..b8426d845a9 --- /dev/null +++ b/benchmark/monetdb/instruction.md @@ -0,0 +1,356 @@ +Go to https://www.monetdb.org/ + +Dowload now. +Latest binary releases. +Ubuntu & Debian. + +https://www.monetdb.org/downloads/deb/ + +Go to the server where you want to install MonetDB. +``` +$ sudo mcedit /etc/apt/sources.list.d/monetdb.list +``` +Write: +``` +deb https://dev.monetdb.org/downloads/deb/ bionic monetdb +``` + +``` +$ wget --output-document=- https://www.monetdb.org/downloads/MonetDB-GPG-KEY | sudo apt-key add - + +$ sudo apt update +$ sudo apt install monetdb5-sql monetdb-client + +$ sudo systemctl enable monetdbd +$ sudo systemctl start monetdbd +$ sudo usermod -a -G monetdb $USER +``` + +Logout and login back to your server. + +Tutorial: +https://www.monetdb.org/Documentation/UserGuide/Tutorial + +Creating the database: + +``` +$ sudo mkdir /opt/monetdb +$ sudo chmod 777 /opt/monetdb +$ monetdbd create /opt/monetdb + +$ monetdbd start /opt/monetdb +cannot remove socket files +``` + +Now you have to stop MonetDB, copy the contents of `/var/monetdb5` to `/opt/monetdb` and replace the `/var/monetdb5` with symlink to `/opt/monetdb`. This is necessary, because I don't have free space in `/var` and creation of database in `/opt` did not succeed. + +Start MonetDB again. + +``` +$ sudo systemctl start monetdbd +``` + +``` +$ monetdb create test +created database in maintenance mode: test + +$ monetdb release test +taken database out of maintenance mode: test +``` + +Run client: +``` +$ mclient -u monetdb -d test +``` + +Type password: monetdb + +``` +CREATE TABLE hits +( + "WatchID" BIGINT, + "JavaEnable" TINYINT, + "Title" TEXT, + "GoodEvent" SMALLINT, + "EventTime" TIMESTAMP, + "EventDate" Date, + "CounterID" INTEGER, + "ClientIP" INTEGER, + "RegionID" INTEGER, + "UserID" BIGINT, + "CounterClass" TINYINT, + "OS" TINYINT, + "UserAgent" TINYINT, + "URL" TEXT, + "Referer" TEXT, + "Refresh" TINYINT, + "RefererCategoryID" SMALLINT, + "RefererRegionID" INTEGER, + "URLCategoryID" SMALLINT, + "URLRegionID" INTEGER, + "ResolutionWidth" SMALLINT, + "ResolutionHeight" SMALLINT, + "ResolutionDepth" TINYINT, + "FlashMajor" TINYINT, + "FlashMinor" TINYINT, + "FlashMinor2" TEXT, + "NetMajor" TINYINT, + "NetMinor" TINYINT, + "UserAgentMajor" SMALLINT, + "UserAgentMinor" TEXT, + "CookieEnable" TINYINT, + "JavascriptEnable" TINYINT, + "IsMobile" TINYINT, + "MobilePhone" TINYINT, + "MobilePhoneModel" TEXT, + "Params" TEXT, + "IPNetworkID" INTEGER, + "TraficSourceID" TINYINT, + "SearchEngineID" SMALLINT, + "SearchPhrase" TEXT, + "AdvEngineID" TINYINT, + "IsArtifical" TINYINT, + "WindowClientWidth" SMALLINT, + "WindowClientHeight" SMALLINT, + "ClientTimeZone" SMALLINT, + "ClientEventTime" TIMESTAMP, + "SilverlightVersion1" TINYINT, + "SilverlightVersion2" TINYINT, + "SilverlightVersion3" INTEGER, + "SilverlightVersion4" SMALLINT, + "PageCharset" TEXT, + "CodeVersion" INTEGER, + "IsLink" TINYINT, + "IsDownload" TINYINT, + "IsNotBounce" TINYINT, + "FUniqID" BIGINT, + "OriginalURL" TEXT, + "HID" INTEGER, + "IsOldCounter" TINYINT, + "IsEvent" TINYINT, + "IsParameter" TINYINT, + "DontCountHits" TINYINT, + "WithHash" TINYINT, + "HitColor" TEXT, + "LocalEventTime" TIMESTAMP, + "Age" TINYINT, + "Sex" TINYINT, + "Income" TINYINT, + "Interests" SMALLINT, + "Robotness" TINYINT, + "RemoteIP" INTEGER, + "WindowName" INTEGER, + "OpenerName" INTEGER, + "HistoryLength" SMALLINT, + "BrowserLanguage" TEXT, + "BrowserCountry" TEXT, + "SocialNetwork" TEXT, + "SocialAction" TEXT, + "HTTPError" SMALLINT, + "SendTiming" INTEGER, + "DNSTiming" INTEGER, + "ConnectTiming" INTEGER, + "ResponseStartTiming" INTEGER, + "ResponseEndTiming" INTEGER, + "FetchTiming" INTEGER, + "SocialSourceNetworkID" TINYINT, + "SocialSourcePage" TEXT, + "ParamPrice" BIGINT, + "ParamOrderID" TEXT, + "ParamCurrency" TEXT, + "ParamCurrencyID" SMALLINT, + "OpenstatServiceName" TEXT, + "OpenstatCampaignID" TEXT, + "OpenstatAdID" TEXT, + "OpenstatSourceID" TEXT, + "UTMSource" TEXT, + "UTMMedium" TEXT, + "UTMCampaign" TEXT, + "UTMContent" TEXT, + "UTMTerm" TEXT, + "FromTag" TEXT, + "HasGCLID" TINYINT, + "RefererHash" BIGINT, + "URLHash" BIGINT, + "CLID" INTEGER +); +``` + +# How to prepare data + +Download the 100 million rows dataset from here and insert into ClickHouse: +https://clickhouse.tech/docs/en/getting-started/example-datasets/metrica/ + +Create the dataset from ClickHouse: + +``` +SELECT + toInt64(WatchID) = -9223372036854775808 ? -9223372036854775807 : toInt64(WatchID), + toInt8(JavaEnable) = -128 ? -127 : toInt8(JavaEnable), + toValidUTF8(toString(Title)), + toInt16(GoodEvent) = -32768 ? -32767 : toInt16(GoodEvent), + EventTime, + EventDate, + toInt32(CounterID) = -2147483648 ? -2147483647 : toInt32(CounterID), + toInt32(ClientIP) = -2147483648 ? -2147483647 : toInt32(ClientIP), + toInt32(RegionID) = -2147483648 ? -2147483647 : toInt32(RegionID), + toInt64(UserID) = -9223372036854775808 ? -9223372036854775807 : toInt64(UserID), + toInt8(CounterClass) = -128 ? -127 : toInt8(CounterClass), + toInt8(OS) = -128 ? -127 : toInt8(OS), + toInt8(UserAgent) = -128 ? -127 : toInt8(UserAgent), + toValidUTF8(toString(URL)), + toValidUTF8(toString(Referer)), + toInt8(Refresh) = -128 ? -127 : toInt8(Refresh), + toInt16(RefererCategoryID) = -32768 ? -32767 : toInt16(RefererCategoryID), + toInt32(RefererRegionID) = -2147483648 ? -2147483647 : toInt32(RefererRegionID), + toInt16(URLCategoryID) = -32768 ? -32767 : toInt16(URLCategoryID), + toInt32(URLRegionID) = -2147483648 ? -2147483647 : toInt32(URLRegionID), + toInt16(ResolutionWidth) = -32768 ? -32767 : toInt16(ResolutionWidth), + toInt16(ResolutionHeight) = -32768 ? -32767 : toInt16(ResolutionHeight), + toInt8(ResolutionDepth) = -128 ? -127 : toInt8(ResolutionDepth), + toInt8(FlashMajor) = -128 ? -127 : toInt8(FlashMajor), + toInt8(FlashMinor) = -128 ? -127 : toInt8(FlashMinor), + toValidUTF8(toString(FlashMinor2)), + toInt8(NetMajor) = -128 ? -127 : toInt8(NetMajor), + toInt8(NetMinor) = -128 ? -127 : toInt8(NetMinor), + toInt16(UserAgentMajor) = -32768 ? -32767 : toInt16(UserAgentMajor), + toValidUTF8(toString(UserAgentMinor)), + toInt8(CookieEnable) = -128 ? -127 : toInt8(CookieEnable), + toInt8(JavascriptEnable) = -128 ? -127 : toInt8(JavascriptEnable), + toInt8(IsMobile) = -128 ? -127 : toInt8(IsMobile), + toInt8(MobilePhone) = -128 ? -127 : toInt8(MobilePhone), + toValidUTF8(toString(MobilePhoneModel)), + toValidUTF8(toString(Params)), + toInt32(IPNetworkID) = -2147483648 ? -2147483647 : toInt32(IPNetworkID), + toInt8(TraficSourceID) = -128 ? -127 : toInt8(TraficSourceID), + toInt16(SearchEngineID) = -32768 ? -32767 : toInt16(SearchEngineID), + toValidUTF8(toString(SearchPhrase)), + toInt8(AdvEngineID) = -128 ? -127 : toInt8(AdvEngineID), + toInt8(IsArtifical) = -128 ? -127 : toInt8(IsArtifical), + toInt16(WindowClientWidth) = -32768 ? -32767 : toInt16(WindowClientWidth), + toInt16(WindowClientHeight) = -32768 ? -32767 : toInt16(WindowClientHeight), + toInt16(ClientTimeZone) = -32768 ? -32767 : toInt16(ClientTimeZone), + ClientEventTime, + toInt8(SilverlightVersion1) = -128 ? -127 : toInt8(SilverlightVersion1), + toInt8(SilverlightVersion2) = -128 ? -127 : toInt8(SilverlightVersion2), + toInt32(SilverlightVersion3) = -2147483648 ? -2147483647 : toInt32(SilverlightVersion3), + toInt16(SilverlightVersion4) = -32768 ? -32767 : toInt16(SilverlightVersion4), + toValidUTF8(toString(PageCharset)), + toInt32(CodeVersion) = -2147483648 ? -2147483647 : toInt32(CodeVersion), + toInt8(IsLink) = -128 ? -127 : toInt8(IsLink), + toInt8(IsDownload) = -128 ? -127 : toInt8(IsDownload), + toInt8(IsNotBounce) = -128 ? -127 : toInt8(IsNotBounce), + toInt64(FUniqID) = -9223372036854775808 ? -9223372036854775807 : toInt64(FUniqID), + toValidUTF8(toString(OriginalURL)), + toInt32(HID) = -2147483648 ? -2147483647 : toInt32(HID), + toInt8(IsOldCounter) = -128 ? -127 : toInt8(IsOldCounter), + toInt8(IsEvent) = -128 ? -127 : toInt8(IsEvent), + toInt8(IsParameter) = -128 ? -127 : toInt8(IsParameter), + toInt8(DontCountHits) = -128 ? -127 : toInt8(DontCountHits), + toInt8(WithHash) = -128 ? -127 : toInt8(WithHash), + toValidUTF8(toString(HitColor)), + LocalEventTime, + toInt8(Age) = -128 ? -127 : toInt8(Age), + toInt8(Sex) = -128 ? -127 : toInt8(Sex), + toInt8(Income) = -128 ? -127 : toInt8(Income), + toInt16(Interests) = -32768 ? -32767 : toInt16(Interests), + toInt8(Robotness) = -128 ? -127 : toInt8(Robotness), + toInt32(RemoteIP) = -2147483648 ? -2147483647 : toInt32(RemoteIP), + toInt32(WindowName) = -2147483648 ? -2147483647 : toInt32(WindowName), + toInt32(OpenerName) = -2147483648 ? -2147483647 : toInt32(OpenerName), + toInt16(HistoryLength) = -32768 ? -32767 : toInt16(HistoryLength), + toValidUTF8(toString(BrowserLanguage)), + toValidUTF8(toString(BrowserCountry)), + toValidUTF8(toString(SocialNetwork)), + toValidUTF8(toString(SocialAction)), + toInt16(HTTPError) = -32768 ? -32767 : toInt16(HTTPError), + toInt32(SendTiming) = -2147483648 ? -2147483647 : toInt32(SendTiming), + toInt32(DNSTiming) = -2147483648 ? -2147483647 : toInt32(DNSTiming), + toInt32(ConnectTiming) = -2147483648 ? -2147483647 : toInt32(ConnectTiming), + toInt32(ResponseStartTiming) = -2147483648 ? -2147483647 : toInt32(ResponseStartTiming), + toInt32(ResponseEndTiming) = -2147483648 ? -2147483647 : toInt32(ResponseEndTiming), + toInt32(FetchTiming) = -2147483648 ? -2147483647 : toInt32(FetchTiming), + toInt8(SocialSourceNetworkID) = -128 ? -127 : toInt8(SocialSourceNetworkID), + toValidUTF8(toString(SocialSourcePage)), + toInt64(ParamPrice) = -9223372036854775808 ? -9223372036854775807 : toInt64(ParamPrice), + toValidUTF8(toString(ParamOrderID)), + toValidUTF8(toString(ParamCurrency)), + toInt16(ParamCurrencyID) = -32768 ? -32767 : toInt16(ParamCurrencyID), + toValidUTF8(toString(OpenstatServiceName)), + toValidUTF8(toString(OpenstatCampaignID)), + toValidUTF8(toString(OpenstatAdID)), + toValidUTF8(toString(OpenstatSourceID)), + toValidUTF8(toString(UTMSource)), + toValidUTF8(toString(UTMMedium)), + toValidUTF8(toString(UTMCampaign)), + toValidUTF8(toString(UTMContent)), + toValidUTF8(toString(UTMTerm)), + toValidUTF8(toString(FromTag)), + toInt8(HasGCLID) = -128 ? -127 : toInt8(HasGCLID), + toInt64(RefererHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(RefererHash), + toInt64(URLHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(URLHash), + toInt32(CLID) = -2147483648 ? -2147483647 : toInt32(CLID) +FROM hits_100m_obfuscated +INTO OUTFILE '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' +FORMAT TSV; +``` + +Note that MonetDB does not support the most negative numbers like -128. And we have to convert them by adding one. +It makes impossible to store the values of 64bit identifiers in BIGINT. +Maybe it's a trick to optimize NULLs? + +Upload the data: + +``` +$ mclient -u monetdb -d test +``` + +Type password: monetdb + +``` +COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' USING DELIMITERS '\t'; +``` + +It takes 28 minutes 02 seconds on a server (Linux Ubuntu, Xeon E5-2560v2, 32 logical CPU, 128 GiB RAM, 8xHDD RAID-5, 40 TB). +It is roughly 60 000 rows per second. + +Validate the data: + +``` +SELECT count(*) FROM hits; +``` + +Create an index: + +``` +CREATE INDEX hits_idx ON hits ("CounterID", "EventDate"); +``` + +(it takes 5 seconds) + +Run the benchmark: + +``` +./benchmark.sh | tee log.txt +``` + +You can find the log in `log.txt` file. + +Postprocess data: + +``` +grep clk log.txt | awk '{ if ($3 == "ms") { print $2 / 1000; } else if ($3 == "sec") { print $2 } else { print } }' +``` + +Then replace values with "min" (minutes) timing manually and save to `tmp.txt`. +Then process to JSON format: + +``` +awk '{ + if (i % 3 == 0) { a = $1 } + else if (i % 3 == 1) { b = $1 } + else if (i % 3 == 2) { c = $1; print "[" a ", " b ", " c "]," }; + ++i; }' < tmp.txt +``` + +And paste to `/website/benchmark/dbms/results/005_monetdb.json` in the repository. diff --git a/benchmark/monetdb/usability.md b/benchmark/monetdb/usability.md index c1f35b6bd4e..1f39b3a434c 100644 --- a/benchmark/monetdb/usability.md +++ b/benchmark/monetdb/usability.md @@ -1205,4 +1205,10 @@ How to prepare the benchmark report: `grep clk log.txt | awk '{ if ($3 == "ms") { print $2 / 1000; } else if ($3 == "sec") { print $2 } else { print } }'` - +``` +awk '{ + if (i % 3 == 0) { a = $1 } + else if (i % 3 == 1) { b = $1 } + else if (i % 3 == 2) { c = $1; print "[" a ", " b ", " c "]," }; + ++i; }' < tmp.txt +``` From 8e92539310fd8596a3e602fe7d0159161b795d59 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed, 12 Aug 2020 07:02:59 +0300 Subject: [PATCH 284/374] Update HTML page --- website/benchmark/dbms/index.html | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/website/benchmark/dbms/index.html b/website/benchmark/dbms/index.html index 16bf92cd052..43eb7ac7531 100644 --- a/website/benchmark/dbms/index.html +++ b/website/benchmark/dbms/index.html @@ -47,9 +47,8 @@ <p>Some additional results (marked as x2, x3, x6) are for clustered setup for comparison. These results are contributed from independent teams and hardware specification may differ.</p> <p> - <strong>Disclaimer!</strong> Some results are significantly outdated: + <strong>Disclaimer!</strong> Some results are outdated: <ul> - <li>Results for MonetDB were obtained in 2013 for version v11.15.11 (Feb2013-SP3)</li> <li>Results for MemSQL were obtained in 2015 for version 3.2.</li> <li>Results for Vertica were obtained in 2015 for version 7.1.1.</li> </ul> From d056bb8273338c890cd8d9ac72fab37a88411f82 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed, 12 Aug 2020 07:15:53 +0300 Subject: [PATCH 285/374] Add new log --- benchmark/monetdb/log.txt | 550 +++++++++++++++++++++----------------- 1 file changed, 298 insertions(+), 252 deletions(-) diff --git a/benchmark/monetdb/log.txt b/benchmark/monetdb/log.txt index 920556019bd..40f480bf216 100644 --- a/benchmark/monetdb/log.txt +++ b/benchmark/monetdb/log.txt @@ -1,295 +1,341 @@ +3 SELECT count(*) FROM hits; -1 tuple -clk: 1.250 ms -1 tuple -clk: 1.215 ms -1 tuple -clk: 1.187 ms +1 tuple +clk: 1.262 ms +1 tuple +clk: 1.420 ms +1 tuple +clk: 1.190 ms +3 SELECT count(*) FROM hits WHERE "AdvEngineID" <> 0; -1 tuple -clk: 1.468 sec -1 tuple -clk: 1.726 sec -1 tuple -clk: 2.369 sec +1 tuple +clk: 1.530 sec +1 tuple +clk: 1.489 sec +1 tuple +clk: 1.490 sec +3 SELECT sum("AdvEngineID"), count(*), avg("ResolutionWidth") FROM hits; -1 tuple -clk: 748.860 ms -1 tuple -clk: 655.206 ms -1 tuple -clk: 686.095 ms +1 tuple +clk: 597.512 ms +1 tuple +clk: 579.383 ms +1 tuple +clk: 598.220 ms +3 SELECT sum("UserID") FROM hits; -clk: 683.854 ms -clk: 702.824 ms -clk: 697.182 ms +overflow in calculation. +clk: 568.003 ms +overflow in calculation. +clk: 554.572 ms +overflow in calculation. +clk: 552.076 ms +3 SELECT COUNT(DISTINCT "UserID") FROM hits; -1 tuple -clk: 8.918 sec -1 tuple -clk: 11.265 sec -1 tuple -clk: 6.662 sec +1 tuple +clk: 6.688 sec +1 tuple +clk: 6.689 sec +1 tuple +clk: 6.652 sec +3 SELECT COUNT(DISTINCT "SearchPhrase") FROM hits; -1 tuple -clk: 15.144 sec -1 tuple -clk: 15.054 sec -1 tuple -clk: 15.074 sec +1 tuple +clk: 15.702 sec +1 tuple +clk: 17.189 sec +1 tuple +clk: 15.514 sec +3 SELECT min("EventDate"), max("EventDate") FROM hits; -1 tuple -clk: 695.901 ms -1 tuple -clk: 671.020 ms -1 tuple -clk: 669.029 ms +1 tuple +clk: 697.770 ms +1 tuple +clk: 711.870 ms +1 tuple +clk: 697.177 ms +3 SELECT "AdvEngineID", count(*) FROM hits WHERE "AdvEngineID" <> 0 GROUP BY "AdvEngineID" ORDER BY count(*) DESC; -18 tuples -clk: 1.425 sec -18 tuples -clk: 1.421 sec -18 tuples -clk: 1.441 sec +18 tuples +clk: 1.536 sec +18 tuples +clk: 1.505 sec +18 tuples +clk: 1.492 sec +3 SELECT "RegionID", COUNT(DISTINCT "UserID") AS u FROM hits GROUP BY "RegionID" ORDER BY u DESC LIMIT 10; -10 tuples -clk: 10.135 sec -10 tuples -clk: 10.278 sec -10 tuples -clk: 9.905 sec +10 tuples +clk: 9.965 sec +10 tuples +clk: 10.106 sec +10 tuples +clk: 10.136 sec +3 SELECT "RegionID", sum("AdvEngineID"), count(*) AS c, avg("ResolutionWidth"), COUNT(DISTINCT "UserID") FROM hits GROUP BY "RegionID" ORDER BY c DESC LIMIT 10; -10 tuples -clk: 8.765 sec -10 tuples -clk: 8.729 sec -10 tuples -clk: 8.504 sec +10 tuples +clk: 8.329 sec +10 tuples +clk: 8.601 sec +10 tuples +clk: 8.039 sec +3 SELECT "MobilePhoneModel", COUNT(DISTINCT "UserID") AS u FROM hits WHERE "MobilePhoneModel" <> '' GROUP BY "MobilePhoneModel" ORDER BY u DESC LIMIT 10; -10 tuples -clk: 3.196 sec -10 tuples -clk: 3.347 sec -10 tuples -clk: 3.276 sec +10 tuples +clk: 3.385 sec +10 tuples +clk: 3.321 sec +10 tuples +clk: 3.326 sec +3 SELECT "MobilePhone", "MobilePhoneModel", COUNT(DISTINCT "UserID") AS u FROM hits WHERE "MobilePhoneModel" <> '' GROUP BY "MobilePhone", "MobilePhoneModel" ORDER BY u DESC LIMIT 10; -10 tuples -clk: 3.360 sec -10 tuples -clk: 3.398 sec -10 tuples -clk: 3.422 sec +10 tuples +clk: 3.510 sec +10 tuples +clk: 3.431 sec +10 tuples +clk: 3.382 sec +3 SELECT "SearchPhrase", count(*) AS c FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; -10 tuples -clk: 10.501 sec -10 tuples -clk: 10.388 sec -10 tuples -clk: 10.305 sec +10 tuples +clk: 10.891 sec +10 tuples +clk: 11.483 sec +10 tuples +clk: 10.352 sec +3 SELECT "SearchPhrase", COUNT(DISTINCT "UserID") AS u FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY u DESC LIMIT 10; -10 tuples -clk: 15.811 sec -10 tuples -clk: 15.463 sec -10 tuples -clk: 15.017 sec +10 tuples +clk: 15.711 sec +10 tuples +clk: 15.444 sec +10 tuples +clk: 15.503 sec +3 SELECT "SearchEngineID", "SearchPhrase", count(*) AS c FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchEngineID", "SearchPhrase" ORDER BY c DESC LIMIT 10; -10 tuples -clk: 11.336 sec -10 tuples -clk: 11.229 sec -10 tuples -clk: 11.069 sec +10 tuples +clk: 11.433 sec +10 tuples +clk: 11.399 sec +10 tuples +clk: 11.285 sec +3 SELECT "UserID", count(*) FROM hits GROUP BY "UserID" ORDER BY count(*) DESC LIMIT 10; -10 tuples -clk: 6.965 sec -10 tuples -clk: 6.884 sec -10 tuples -clk: 6.907 sec +10 tuples +clk: 7.184 sec +10 tuples +clk: 7.015 sec +10 tuples +clk: 6.849 sec +3 SELECT "UserID", "SearchPhrase", count(*) FROM hits GROUP BY "UserID", "SearchPhrase" ORDER BY count(*) DESC LIMIT 10; -10 tuples -clk: 28.985 sec -10 tuples -clk: 29.714 sec -10 tuples -clk: 28.438 sec +10 tuples +clk: 29.096 sec +10 tuples +clk: 28.328 sec +10 tuples +clk: 29.247 sec +3 SELECT "UserID", "SearchPhrase", count(*) FROM hits GROUP BY "UserID", "SearchPhrase" LIMIT 10; -10 tuples -clk: 27.820 sec -10 tuples -clk: 29.318 sec -10 tuples -clk: 29.515 sec +10 tuples +clk: 29.457 sec +10 tuples +clk: 29.364 sec +10 tuples +clk: 29.269 sec +3 SELECT "UserID", extract(minute FROM "EventTime") AS m, "SearchPhrase", count(*) FROM hits GROUP BY "UserID", m, "SearchPhrase" ORDER BY count(*) DESC LIMIT 10; -10 tuples -clk: 46.359 sec -10 tuples -clk: 46.382 sec -10 tuples -clk: 46.353 sec +10 tuples +clk: 47.141 sec +10 tuples +clk: 46.495 sec +10 tuples +clk: 46.472 sec +3 SELECT "UserID" FROM hits WHERE "UserID" = -6101065172474983726; -0 tuples -clk: 750.670 ms -0 tuples -clk: 768.232 ms -0 tuples -clk: 765.361 ms +0 tuples +clk: 783.332 ms +0 tuples +clk: 771.157 ms +0 tuples +clk: 783.082 ms +3 SELECT count(*) FROM hits WHERE "URL" LIKE '%metrika%'; -1 tuple -clk: 3.938 sec -1 tuple -clk: 3.932 sec -1 tuple -clk: 3.915 sec +1 tuple +clk: 3.963 sec +1 tuple +clk: 3.930 sec +1 tuple +clk: 3.964 sec +3 SELECT "SearchPhrase", min("URL"), count(*) AS c FROM hits WHERE "URL" LIKE '%metrika%' AND "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; -10 tuples -clk: 3.742 sec -10 tuples -clk: 3.784 sec -10 tuples -clk: 3.727 sec +10 tuples +clk: 3.925 sec +10 tuples +clk: 3.817 sec +10 tuples +clk: 3.802 sec +3 SELECT "SearchPhrase", min("URL"), min("Title"), count(*) AS c, COUNT(DISTINCT "UserID") FROM hits WHERE "Title" LIKE '%Яндекс%' AND "URL" NOT LIKE '%.yandex.%' AND "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; -10 tuples -clk: 6.024 sec -10 tuples -clk: 5.948 sec -10 tuples -clk: 6.001 sec +10 tuples +clk: 6.067 sec +10 tuples +clk: 6.120 sec +10 tuples +clk: 6.012 sec +3 SELECT * FROM hits WHERE "URL" LIKE '%metrika%' ORDER BY "EventTime" LIMIT 10; -10 tuples !87 columns dropped, 29 fields truncated! -clk: 4.130 sec -10 tuples !87 columns dropped, 29 fields truncated! -clk: 4.138 sec -10 tuples !87 columns dropped, 29 fields truncated! -clk: 4.270 sec +10 tuples !87 columns dropped, 29 fields truncated! +clk: 4.251 sec +10 tuples !87 columns dropped, 29 fields truncated! +clk: 4.190 sec +10 tuples !87 columns dropped, 29 fields truncated! +clk: 4.379 sec +3 SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "EventTime" LIMIT 10; -10 tuples -clk: 6.659 sec -10 tuples -clk: 6.751 sec -10 tuples -clk: 6.674 sec +10 tuples +clk: 6.699 sec +10 tuples +clk: 6.718 sec +10 tuples +clk: 6.802 sec +3 SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "SearchPhrase" LIMIT 10; -10 tuples -clk: 6.817 sec -10 tuples -clk: 6.745 sec -10 tuples -clk: 6.793 sec +10 tuples +clk: 6.887 sec +10 tuples +clk: 6.838 sec +10 tuples +clk: 6.844 sec +3 SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "EventTime", "SearchPhrase" LIMIT 10; -10 tuples -clk: 6.742 sec -10 tuples -clk: 6.673 sec -10 tuples -clk: 6.743 sec +10 tuples +clk: 6.806 sec +10 tuples +clk: 6.878 sec +10 tuples +clk: 6.807 sec +3 SELECT "CounterID", avg(length("URL")) AS l, count(*) AS c FROM hits WHERE "URL" <> '' GROUP BY "CounterID" HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -25 tuples -clk: 57.559 sec -25 tuples -clk: 56.546 sec -25 tuples -clk: 58.124 sec +25 tuples +clk: 1:01 min +25 tuples +clk: 55.553 sec +25 tuples +clk: 56.188 sec +3 SELECT sys.getdomain("Referer") AS key, avg(length("Referer")) AS l, count(*) AS c, min("Referer") FROM hits WHERE "Referer" <> '' GROUP BY key HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; -clk: 1:00 min -clk: 59.189 sec -clk: 1:00 min +clk: 1:00 min +clk: 1:00 min +clk: 1:00 min +3 SELECT sum("ResolutionWidth"), sum("ResolutionWidth" + 1), sum("ResolutionWidth" + 2), sum("ResolutionWidth" + 3), sum("ResolutionWidth" + 4), sum("ResolutionWidth" + 5), sum("ResolutionWidth" + 6), sum("ResolutionWidth" + 7), sum("ResolutionWidth" + 8), sum("ResolutionWidth" + 9), sum("ResolutionWidth" + 10), sum("ResolutionWidth" + 11), sum("ResolutionWidth" + 12), sum("ResolutionWidth" + 13), sum("ResolutionWidth" + 14), sum("ResolutionWidth" + 15), sum("ResolutionWidth" + 16), sum("ResolutionWidth" + 17), sum("ResolutionWidth" + 18), sum("ResolutionWidth" + 19), sum("ResolutionWidth" + 20), sum("ResolutionWidth" + 21), sum("ResolutionWidth" + 22), sum("ResolutionWidth" + 23), sum("ResolutionWidth" + 24), sum("ResolutionWidth" + 25), sum("ResolutionWidth" + 26), sum("ResolutionWidth" + 27), sum("ResolutionWidth" + 28), sum("ResolutionWidth" + 29), sum("ResolutionWidth" + 30), sum("ResolutionWidth" + 31), sum("ResolutionWidth" + 32), sum("ResolutionWidth" + 33), sum("ResolutionWidth" + 34), sum("ResolutionWidth" + 35), sum("ResolutionWidth" + 36), sum("ResolutionWidth" + 37), sum("ResolutionWidth" + 38), sum("ResolutionWidth" + 39), sum("ResolutionWidth" + 40), sum("ResolutionWidth" + 41), sum("ResolutionWidth" + 42), sum("ResolutionWidth" + 43), sum("ResolutionWidth" + 44), sum("ResolutionWidth" + 45), sum("ResolutionWidth" + 46), sum("ResolutionWidth" + 47), sum("ResolutionWidth" + 48), sum("ResolutionWidth" + 49), sum("ResolutionWidth" + 50), sum("ResolutionWidth" + 51), sum("ResolutionWidth" + 52), sum("ResolutionWidth" + 53), sum("ResolutionWidth" + 54), sum("ResolutionWidth" + 55), sum("ResolutionWidth" + 56), sum("ResolutionWidth" + 57), sum("ResolutionWidth" + 58), sum("ResolutionWidth" + 59), sum("ResolutionWidth" + 60), sum("ResolutionWidth" + 61), sum("ResolutionWidth" + 62), sum("ResolutionWidth" + 63), sum("ResolutionWidth" + 64), sum("ResolutionWidth" + 65), sum("ResolutionWidth" + 66), sum("ResolutionWidth" + 67), sum("ResolutionWidth" + 68), sum("ResolutionWidth" + 69), sum("ResolutionWidth" + 70), sum("ResolutionWidth" + 71), sum("ResolutionWidth" + 72), sum("ResolutionWidth" + 73), sum("ResolutionWidth" + 74), sum("ResolutionWidth" + 75), sum("ResolutionWidth" + 76), sum("ResolutionWidth" + 77), sum("ResolutionWidth" + 78), sum("ResolutionWidth" + 79), sum("ResolutionWidth" + 80), sum("ResolutionWidth" + 81), sum("ResolutionWidth" + 82), sum("ResolutionWidth" + 83), sum("ResolutionWidth" + 84), sum("ResolutionWidth" + 85), sum("ResolutionWidth" + 86), sum("ResolutionWidth" + 87), sum("ResolutionWidth" + 88), sum("ResolutionWidth" + 89) FROM hits; -1 tuple !77 columns dropped! -clk: 6.374 sec -1 tuple !77 columns dropped! -clk: 6.428 sec -1 tuple !77 columns dropped! -clk: 6.115 sec +1 tuple !77 columns dropped! +clk: 6.221 sec +1 tuple !77 columns dropped! +clk: 6.170 sec +1 tuple !77 columns dropped! +clk: 6.382 sec +3 SELECT "SearchEngineID", "ClientIP", count(*) AS c, sum("Refresh"), avg("ResolutionWidth") FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchEngineID", "ClientIP" ORDER BY c DESC LIMIT 10; -10 tuples -clk: 5.336 sec -10 tuples -clk: 5.555 sec -10 tuples -clk: 5.342 sec +10 tuples +clk: 5.684 sec +10 tuples +clk: 5.585 sec +10 tuples +clk: 5.463 sec +3 SELECT "WatchID", "ClientIP", count(*) AS c, sum("Refresh"), avg("ResolutionWidth") FROM hits WHERE "SearchPhrase" <> '' GROUP BY "WatchID", "ClientIP" ORDER BY c DESC LIMIT 10; -10 tuples -clk: 6.184 sec -10 tuples -clk: 6.252 sec -10 tuples -clk: 6.541 sec +10 tuples +clk: 6.281 sec +10 tuples +clk: 6.574 sec +10 tuples +clk: 6.243 sec +3 SELECT "WatchID", "ClientIP", count(*) AS c, sum("Refresh"), avg("ResolutionWidth") FROM hits GROUP BY "WatchID", "ClientIP" ORDER BY c DESC LIMIT 10; -10 tuples -clk: 43.900 sec -10 tuples -clk: 42.386 sec -10 tuples -clk: 43.674 sec +10 tuples +clk: 44.641 sec +10 tuples +clk: 41.904 sec +10 tuples +clk: 43.218 sec +3 SELECT "URL", count(*) AS c FROM hits GROUP BY "URL" ORDER BY c DESC LIMIT 10; -10 tuples -clk: 1:25 min -10 tuples -clk: 1:23 min -10 tuples -clk: 1:23 min +10 tuples +clk: 1:24 min +10 tuples +clk: 1:31 min +10 tuples +clk: 1:24 min +3 SELECT 1, "URL", count(*) AS c FROM hits GROUP BY 1, "URL" ORDER BY c DESC LIMIT 10; -10 tuples -clk: 1:23 min -10 tuples -clk: 1:22 min -10 tuples -clk: 1:23 min +10 tuples +clk: 1:24 min +10 tuples +clk: 1:25 min +10 tuples +clk: 1:24 min +3 SELECT "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3, count(*) AS c FROM hits GROUP BY "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3 ORDER BY c DESC LIMIT 10; -10 tuples -clk: 26.289 sec -10 tuples -clk: 25.788 sec -10 tuples -clk: 24.693 sec +10 tuples +clk: 26.438 sec +10 tuples +clk: 26.033 sec +10 tuples +clk: 26.147 sec +3 SELECT "URL", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "DontCountHits" = 0 AND "Refresh" = 0 AND "URL" <> '' GROUP BY "URL" ORDER BY "PageViews" DESC LIMIT 10; -10 tuples -clk: 4.506 sec -10 tuples -clk: 4.503 sec -10 tuples -clk: 4.496 sec +10 tuples +clk: 4.825 sec +10 tuples +clk: 4.618 sec +10 tuples +clk: 4.623 sec +3 SELECT "Title", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "DontCountHits" = 0 AND "Refresh" = 0 AND "Title" <> '' GROUP BY "Title" ORDER BY "PageViews" DESC LIMIT 10; -10 tuples -clk: 4.306 sec -10 tuples -clk: 4.337 sec -10 tuples -clk: 4.355 sec +10 tuples +clk: 4.380 sec +10 tuples +clk: 4.418 sec +10 tuples +clk: 4.413 sec +3 SELECT "URL", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 AND "IsLink" <> 0 AND "IsDownload" = 0 GROUP BY "URL" ORDER BY "PageViews" DESC LIMIT 1000; -1000 tuples -clk: 4.274 sec -1000 tuples -clk: 4.145 sec -1000 tuples -clk: 4.305 sec +1000 tuples +clk: 4.259 sec +1000 tuples +clk: 4.195 sec +1000 tuples +clk: 4.195 sec +3 SELECT "TraficSourceID", "SearchEngineID", "AdvEngineID", CASE WHEN ("SearchEngineID" = 0 AND "AdvEngineID" = 0) THEN "Referer" ELSE '' END AS Src, "URL" AS Dst, count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 GROUP BY "TraficSourceID", "SearchEngineID", "AdvEngineID", CASE WHEN ("SearchEngineID" = 0 AND "AdvEngineID" = 0) THEN "Referer" ELSE '' END, "URL" ORDER BY "PageViews" DESC LIMIT 1000; -1000 tuples -clk: 3.175 sec -1000 tuples -clk: 3.121 sec -1000 tuples -clk: 3.186 sec +1000 tuples +clk: 3.233 sec +1000 tuples +clk: 3.180 sec +1000 tuples +clk: 3.181 sec +3 SELECT "URLHash", "EventDate", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 AND "TraficSourceID" IN (-1, 6) AND "RefererHash" = 686716256552154761 GROUP BY "URLHash", "EventDate" ORDER BY "PageViews" DESC LIMIT 100; -0 tuples -clk: 2.600 sec -0 tuples -clk: 2.598 sec -0 tuples -clk: 2.507 sec +0 tuples +clk: 2.656 sec +0 tuples +clk: 2.557 sec +0 tuples +clk: 2.561 sec +3 SELECT "WindowClientWidth", "WindowClientHeight", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 AND "DontCountHits" = 0 AND "URLHash" = 686716256552154761 GROUP BY "WindowClientWidth", "WindowClientHeight" ORDER BY "PageViews" DESC LIMIT 10000; -0 tuples -clk: 4.239 sec -0 tuples -clk: 4.205 sec -0 tuples -clk: 4.166 sec +0 tuples +clk: 4.161 sec +0 tuples +clk: 4.243 sec +0 tuples +clk: 4.166 sec +3 SELECT DATE_TRUNC('minute', "EventTime") AS "Minute", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-02' AND "Refresh" = 0 AND "DontCountHits" = 0 GROUP BY DATE_TRUNC('minute', "EventTime") ORDER BY DATE_TRUNC('minute', "EventTime"); -0 tuples -clk: 4.109 sec -0 tuples -clk: 4.222 sec -0 tuples -clk: 4.209 sec +0 tuples +clk: 4.199 sec +0 tuples +clk: 4.211 sec +0 tuples +clk: 4.190 sec From bc86f74a5b347e974db9d8ba1bf19b973fa839fc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed, 12 Aug 2020 07:30:26 +0300 Subject: [PATCH 286/374] Update results --- benchmark/monetdb/instruction.md | 2 +- .../benchmark/dbms/results/005_monetdb.json | 86 +++++++++---------- 2 files changed, 44 insertions(+), 44 deletions(-) diff --git a/benchmark/monetdb/instruction.md b/benchmark/monetdb/instruction.md index b8426d845a9..482693d6485 100644 --- a/benchmark/monetdb/instruction.md +++ b/benchmark/monetdb/instruction.md @@ -339,7 +339,7 @@ You can find the log in `log.txt` file. Postprocess data: ``` -grep clk log.txt | awk '{ if ($3 == "ms") { print $2 / 1000; } else if ($3 == "sec") { print $2 } else { print } }' +grep clk log.txt | tr -d '\r' | awk '{ if ($3 == "ms") { print $2 / 1000; } else if ($3 == "sec") { print $2 } else { print } }' ``` Then replace values with "min" (minutes) timing manually and save to `tmp.txt`. diff --git a/website/benchmark/dbms/results/005_monetdb.json b/website/benchmark/dbms/results/005_monetdb.json index a29186f0922..c378bb0a4fa 100644 --- a/website/benchmark/dbms/results/005_monetdb.json +++ b/website/benchmark/dbms/results/005_monetdb.json @@ -7,49 +7,49 @@ "comments": "", "result": [ - [0.00125, 0.001215, 0.001187], - [1.468, 1.726, 2.369], - [0.74886, 0.655206, 0.686095], - [0.683854, 0.702824, 0.697182], - [8.918, 11.265, 6.662], - [15.144, 15.054, 15.074], - [0.695901, 0.67102, 0.669029], - [1.425, 1.421, 1.441], - [10.135, 10.278, 9.905], - [8.765, 8.729, 8.504], - [3.196, 3.347, 3.276], - [3.360, 3.398, 3.422], - [10.501, 10.388, 10.305], - [15.811, 15.463, 15.017], - [11.336, 11.229, 11.069], - [6.965, 6.884, 6.907], - [28.985, 29.714, 28.438], - [27.820, 29.318, 29.515], - [46.359, 46.382, 46.353], - [0.75067, 0.768232, 0.765361], - [3.938, 3.932, 3.915], - [3.742, 3.784, 3.727], - [6.024, 5.948, 6.001], - [4.130, 4.138, 4.270], - [6.659, 6.751, 6.674], - [6.817, 6.745, 6.793], - [6.742, 6.673, 6.743], - [57.559, 56.546, 58.124], - [60, 59.189, 60], - [6.374, 6.428, 6.115], - [5.336, 5.555, 5.342], - [6.184, 6.252, 6.541], - [43.900, 42.386, 43.674], - [85, 83, 83], - [83, 82, 83], - [26.289, 25.788, 24.693], - [4.506, 4.503, 4.496], - [4.306, 4.337, 4.355], - [4.274, 4.145, 4.305], - [3.175, 3.121, 3.186], - [2.600, 2.598, 2.507], - [4.239, 4.205, 4.166], - [4.109, 4.222, 4.209] + [0.001262, 0.00142, 0.00119], + [1.530, 1.489, 1.490], + [0.597512, 0.579383, 0.59822], + [0.568003, 0.554572, 0.552076], + [6.688, 6.689, 6.652], + [15.702, 17.189, 15.514], + [0.69777, 0.71187, 0.697177], + [1.536, 1.505, 1.492], + [9.965, 10.106, 10.136], + [8.329, 8.601, 8.039], + [3.385, 3.321, 3.326], + [3.510, 3.431, 3.382], + [10.891, 11.483, 10.352], + [15.711, 15.444, 15.503], + [11.433, 11.399, 11.285], + [7.184, 7.015, 6.849], + [29.096, 28.328, 29.247], + [29.457, 29.364, 29.269], + [47.141, 46.495, 46.472], + [0.783332, 0.771157, 0.783082], + [3.963, 3.930, 3.964], + [3.925, 3.817, 3.802], + [6.067, 6.120, 6.012], + [4.251, 4.190, 4.379], + [6.699, 6.718, 6.802], + [6.887, 6.838, 6.844], + [6.806, 6.878, 6.807], + [61, 55.553, 56.188], + [60, 60, 60], + [6.221, 6.170, 6.382], + [5.684, 5.585, 5.463], + [6.281, 6.574, 6.243], + [44.641, 41.904, 43.218], + [84, 91, 84], + [84, 85, 84], + [26.438, 26.033, 26.147], + [4.825, 4.618, 4.623], + [4.380, 4.418, 4.413], + [4.259, 4.195, 4.195], + [3.233, 3.180, 3.181], + [2.656, 2.557, 2.561], + [4.161, 4.243, 4.166], + [4.199, 4.211, 4.190] ] } ] From 89b712a180aaafabc3a6a70e4fce965fa53e2dd0 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <akuzm@yandex-team.ru> Date: Wed, 12 Aug 2020 08:39:29 +0300 Subject: [PATCH 287/374] performance comparison --- docker/test/performance-comparison/perf.py | 33 +++++++++++++++------- 1 file changed, 23 insertions(+), 10 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 2a2b8ee679e..a659326b068 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -222,17 +222,22 @@ for query_index, q in enumerate(test_queries): query_error_on_connection[conn_index] = traceback.format_exc(); continue - # If prewarm fails for the query on both servers -- report the error, skip - # the query and continue testing the next query. - if query_error_on_connection.count(None) == 0: - print(query_error_on_connection[0], file = sys.stderr) - continue + # Report all errors that ocurred during prewarm and decide what to do next. + # If prewarm fails for the query on all servers -- skip the query and + # continue testing the next query. # If prewarm fails on one of the servers, run the query on the rest of them. - # Useful for queries that use new functions added in the new server version. - if query_error_on_connection.count(None) < len(query_error_on_connection): - no_error = [i for i, e in enumerate(query_error_on_connection) if not e] - print(f'partial\t{query_index}\t{no_error}') + no_errors = [] + for i, e in enumerate(query_error_on_connection): + if e: + print(e, file = sys.stderr) + else: + no_errors.append(i) + + if len(no_errors) == 0: + continue + elif len(no_errors) < len(connections): + print(f'partial\t{query_index}\t{no_errors}') # Now, perform measured runs. # Track the time spent by the client to process this query, so that we can @@ -245,7 +250,15 @@ for query_index, q in enumerate(test_queries): for conn_index, c in enumerate(connections): if query_error_on_connection[conn_index]: continue - res = c.execute(q, query_id = run_id) + + try: + res = c.execute(q, query_id = run_id) + except Exception as e: + # Add query id to the exception to make debugging easier. + e.args = (run_id, *e.args) + e.message = run_id + ': ' + e.message + raise + print(f'query\t{query_index}\t{run_id}\t{conn_index}\t{c.last_query.elapsed}') server_seconds += c.last_query.elapsed From c31f4f013a02f5efc44326815365f2e80413c14b Mon Sep 17 00:00:00 2001 From: BohuTANG <overred.shuttler@gmail.com> Date: Wed, 12 Aug 2020 13:50:22 +0800 Subject: [PATCH 288/374] Add MergeTree Write-Ahead-Log(WAL) dump tool --- utils/CMakeLists.txt | 1 + utils/wal-dump/CMakeLists.txt | 2 + utils/wal-dump/main.cpp | 78 +++++++++++++++++++++++++++++++++++ 3 files changed, 81 insertions(+) create mode 100644 utils/wal-dump/CMakeLists.txt create mode 100644 utils/wal-dump/main.cpp diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 94042ea4090..0dd95388e7d 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -28,6 +28,7 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) add_subdirectory (test-data-generator) add_subdirectory (convert-month-partitioned-parts) add_subdirectory (checksum-for-compressed-block) + add_subdirectory (wal-dump) endif () if (ENABLE_CODE_QUALITY) diff --git a/utils/wal-dump/CMakeLists.txt b/utils/wal-dump/CMakeLists.txt new file mode 100644 index 00000000000..980253b89c7 --- /dev/null +++ b/utils/wal-dump/CMakeLists.txt @@ -0,0 +1,2 @@ +add_executable (wal-dump main.cpp) +target_link_libraries(wal-dump PRIVATE dbms boost::program_options) diff --git a/utils/wal-dump/main.cpp b/utils/wal-dump/main.cpp new file mode 100644 index 00000000000..361aa9df887 --- /dev/null +++ b/utils/wal-dump/main.cpp @@ -0,0 +1,78 @@ +#include <iostream> + +#include <boost/program_options.hpp> + +#include <Compression/CompressedReadBuffer.h> +#include <Compression/CompressedReadBufferFromFile.h> +#include <Compression/CompressedWriteBuffer.h> +#include <DataStreams/NativeBlockInputStream.h> +#include <IO/Operators.h> +#include <IO/ReadBufferFromFile.h> +#include <IO/ReadHelpers.h> +#include <IO/WriteBufferFromFileDescriptor.h> +#include <Storages/MergeTree/MergeTreeWriteAheadLog.h> + +/* + * Dump the Write Ahead Log file, outputs: + * Part 0, Version: 0, Action : ADD_PART, Name: 4_1_1_0, Block: + a Int32 Int32(size = 2), b Int32 Int32(size = 2), c Int32 Int32(size = 2) + */ + +static void dump(const std::string & bin_path) +{ + DB::ReadBufferFromFile in(bin_path); + DB::NativeBlockInputStream block_in(in, 0); + DB::Block block; + + DB::WriteBufferFromFileDescriptor out(STDOUT_FILENO); + + for (size_t part_num = 0; !in.eof(); ++part_num) + { + UInt8 version; + String part_name; + DB::MergeTreeWriteAheadLog::ActionType action_type; + + DB::readIntBinary(version, in); + DB::readIntBinary(action_type, in); + DB::readStringBinary(part_name, in); + block = block_in.read(); + + out << "Part " << part_num << ", Version: " << version + << ", Action : " << (action_type == DB::MergeTreeWriteAheadLog::ActionType::ADD_PART ? "ADD_PART" : "DROP_PART") + << ", Name: " << part_name << ", Block:\n"; + out << block.dumpStructure() << "\n"; + out << "\n" << DB::flush; + } +} + + +int main(int argc, char ** argv) +{ + boost::program_options::options_description desc("Allowed options"); + desc.add_options()("help,h", "produce help message"); + + boost::program_options::variables_map options; + boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); + + if (options.count("help") || argc != 2) + { + std::cout << "Usage: " << argv[0] << " wal.bin" << std::endl; + std::cout << desc << std::endl; + return 1; + } + + try + { + dump(argv[1]); + } + catch (const DB::Exception & e) + { + std::cerr << e.what() << ", " << e.message() << std::endl + << std::endl + << "Stack trace:" << std::endl + << e.getStackTraceString() << std::endl; + throw; + } + + return 0; +} From adf84b19c09066ce4a40bd1406097496f8f971f9 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Wed, 12 Aug 2020 07:19:28 +0000 Subject: [PATCH 289/374] Bump pymdown-extensions from 7.1 to 8.0 in /docs/tools Bumps [pymdown-extensions](https://github.com/facelessuser/pymdown-extensions) from 7.1 to 8.0. - [Release notes](https://github.com/facelessuser/pymdown-extensions/releases) - [Commits](https://github.com/facelessuser/pymdown-extensions/compare/7.1...8.0) Signed-off-by: dependabot-preview[bot] <support@dependabot.com> --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 7ace18b639c..03fa9fbb12c 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -24,7 +24,7 @@ nose==1.3.7 protobuf==3.12.4 numpy==1.19.1 Pygments==2.5.2 -pymdown-extensions==7.1 +pymdown-extensions==8.0 python-slugify==4.0.1 PyYAML==5.3.1 repackage==0.7.3 From 6097f769c7cc2c13b380ad869de5bf92f0615455 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin <a3at.mail@gmail.com> Date: Tue, 11 Aug 2020 03:18:43 +0300 Subject: [PATCH 290/374] Fix assertion for carriage return in the header in TSVWithNames* format. TabSeparatedRowInputFormat::readPrefix() tries to check for carriage return (via checkForCarriageReturn()), however it does the check after checking the column is exists, reorder the codepath a little bit to run checkForCarriageReturn() before. --- .../Impl/TabSeparatedRowInputFormat.cpp | 18 +++++++++++++----- .../01406_carriage_return_in_tsv_csv.reference | 9 +++++++++ .../01406_carriage_return_in_tsv_csv.sh | 17 +++++++++++++++++ 3 files changed, 39 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01406_carriage_return_in_tsv_csv.reference create mode 100755 tests/queries/0_stateless/01406_carriage_return_in_tsv_csv.sh diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 8596cb850d9..b93d22d1f41 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -38,7 +38,7 @@ static void checkForCarriageReturn(ReadBuffer & in) throw Exception("\nYou have carriage return (\\r, 0x0D, ASCII 13) at end of first row." "\nIt's like your input data has DOS/Windows style line separators, that are illegal in TabSeparated format." " You must transform your file to Unix format." - "\nBut if you really need carriage return at end of string value of last column, you need to escape it as \\r.", + "\nBut if you really need carriage return at end of string value of last column, you need to escape it as \\\\r.", ErrorCodes::INCORRECT_DATA); } @@ -140,16 +140,24 @@ void TabSeparatedRowInputFormat::readPrefix() if (format_settings.with_names_use_header) { String column_name; - do + for (;;) { readEscapedString(column_name, in); - addInputColumn(column_name); + if (!checkChar('\t', in)) + { + /// Check last column for \r before adding it, otherwise an error will be: + /// "Unknown field found in TSV header" + checkForCarriageReturn(in); + addInputColumn(column_name); + break; + } + else + addInputColumn(column_name); } - while (checkChar('\t', in)); + if (!in.eof()) { - checkForCarriageReturn(in); assertChar('\n', in); } } diff --git a/tests/queries/0_stateless/01406_carriage_return_in_tsv_csv.reference b/tests/queries/0_stateless/01406_carriage_return_in_tsv_csv.reference new file mode 100644 index 00000000000..bdd25a5ec95 --- /dev/null +++ b/tests/queries/0_stateless/01406_carriage_return_in_tsv_csv.reference @@ -0,0 +1,9 @@ +CSVWithNames +bar +TSVWithNames +117 + +TSV +117 +key\r +bar diff --git a/tests/queries/0_stateless/01406_carriage_return_in_tsv_csv.sh b/tests/queries/0_stateless/01406_carriage_return_in_tsv_csv.sh new file mode 100755 index 00000000000..b0bdb4627ee --- /dev/null +++ b/tests/queries/0_stateless/01406_carriage_return_in_tsv_csv.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +echo 'CSVWithNames' +echo -n $'key\r\nbar\r\n' | ${CLICKHOUSE_LOCAL} --input-format CSVWithNames -S 'key String' -q 'select * from table' + +echo 'TSVWithNames' +echo -n $'key\r\nbar\r\n' | ${CLICKHOUSE_LOCAL} --input-format TSVWithNames -S 'key String' -q 'select * from table' >& /dev/null +echo $? +echo -n $'key\\r\nbar\n' | ${CLICKHOUSE_LOCAL} --input_format_skip_unknown_fields=1 --input-format TSVWithNames -S 'key String' -q 'select * from table' + +echo 'TSV' +echo -n $'key\r\nbar\r\n' | ${CLICKHOUSE_LOCAL} --input-format TSV -S 'key String' -q 'select * from table' >& /dev/null +echo $? +echo -n $'key\\r\nbar\n' | ${CLICKHOUSE_LOCAL} --input-format TSV -S 'key String' -q 'select * from table' From 6fdfe65694876a279bb108012542eb8fedf70cb1 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <akuzm@yandex-team.ru> Date: Wed, 12 Aug 2020 11:54:46 +0300 Subject: [PATCH 291/374] performance comparison --- docker/test/performance-comparison/report.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 31817e0ea10..fa0030d5252 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -353,7 +353,7 @@ if args.report == 'main': attrs[1] = '' if float(row[0]) > allowed_single_run_time: attrs[0] = f'style="background: {color_bad}"' - errors_explained.append([f'<a href="#{anchor}">The query no. {row[3]} of test \'{row[2]}\' is taking too long to run. Keep the run time below {allowed_single_run} seconds"</a>']) + errors_explained.append([f'<a href="#{anchor}">The query no. {row[3]} of test \'{row[2]}\' is taking too long to run. Keep the run time below {allowed_single_run_time} seconds"</a>']) slow_average_tests += 1 else: attrs[0] = '' From e6420c82ca0845bc0987027cf7329c05c937eb70 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Aug 2020 18:04:36 +0300 Subject: [PATCH 292/374] mysql/postgresql: move Dockerfiles and docker_compose to docker/test (#13448) * mysql/postgresql: move Dockerfiles and docker_compose to docker/test * Build client containers in CI * pass tag to docker_compose --- docker/images.json | 20 ++++++++++++++++++ .../mysql_golang_client}/Dockerfile | 3 +++ .../integration/mysql_golang_client}/main.go | 0 .../integration/mysql_java_client}/Dockerfile | 3 +++ .../integration/mysql_java_client}/Test.java | 0 .../integration/mysql_js_client/Dockerfile | 8 +++++++ .../test/integration/mysql_js_client}/test.js | 0 .../integration/mysql_php_client}/Dockerfile | 3 +++ .../integration/mysql_php_client}/client.crt | 0 .../integration/mysql_php_client}/client.key | 0 .../integration/mysql_php_client}/test.php | 0 .../mysql_php_client}/test_ssl.php | 0 .../postgresql_java_client}/Dockerfile | 3 +++ .../postgresql_java_client}/Test.java | 0 .../compose/docker_compose_mysql_client.yml | 0 .../docker_compose_mysql_golang_client.yml | 4 +--- .../docker_compose_mysql_java_client.yml | 4 +--- .../docker_compose_mysql_js_client.yml | 4 +--- .../docker_compose_mysql_php_client.yml | 3 +-- .../compose/docker_compose_postgesql.yml | 0 .../docker_compose_postgesql_java_client.yml | 4 +--- .../integration/runner/dockerd-entrypoint.sh | 6 ++++++ tests/integration/runner | 21 ++++++++++++++++++- .../clients/mysqljs/Dockerfile | 5 ----- .../golang/0.reference => golang.reference} | 0 .../java/0.reference => java.reference} | 0 tests/integration/test_mysql_protocol/test.py | 17 ++++++++------- .../java/0.reference => java.reference} | 0 .../test_postgresql_protocol/test.py | 9 ++++---- 29 files changed, 85 insertions(+), 32 deletions(-) rename {tests/integration/test_mysql_protocol/clients/golang => docker/test/integration/mysql_golang_client}/Dockerfile (52%) rename {tests/integration/test_mysql_protocol/clients/golang => docker/test/integration/mysql_golang_client}/main.go (100%) rename {tests/integration/test_mysql_protocol/clients/java => docker/test/integration/mysql_java_client}/Dockerfile (84%) rename {tests/integration/test_mysql_protocol/clients/java => docker/test/integration/mysql_java_client}/Test.java (100%) create mode 100644 docker/test/integration/mysql_js_client/Dockerfile rename {tests/integration/test_mysql_protocol/clients/mysqljs => docker/test/integration/mysql_js_client}/test.js (100%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/Dockerfile (65%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/client.crt (100%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/client.key (100%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/test.php (100%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/test_ssl.php (100%) rename {tests/integration/test_postgresql_protocol/clients/java => docker/test/integration/postgresql_java_client}/Dockerfile (82%) rename {tests/integration/test_postgresql_protocol/clients/java => docker/test/integration/postgresql_java_client}/Test.java (100%) rename tests/integration/test_mysql_protocol/clients/mysql/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_client.yml (100%) rename tests/integration/test_mysql_protocol/clients/golang/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml (53%) rename tests/integration/test_mysql_protocol/clients/java/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml (54%) rename tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml (56%) rename tests/integration/test_mysql_protocol/clients/php-mysqlnd/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml (54%) rename tests/integration/test_postgresql_protocol/clients/psql/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_postgesql.yml (100%) rename tests/integration/test_postgresql_protocol/clients/java/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml (51%) delete mode 100644 tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile rename tests/integration/test_mysql_protocol/{clients/golang/0.reference => golang.reference} (100%) rename tests/integration/test_mysql_protocol/{clients/java/0.reference => java.reference} (100%) rename tests/integration/test_postgresql_protocol/{clients/java/0.reference => java.reference} (100%) diff --git a/docker/images.json b/docker/images.json index 09114cc9710..0c4307f8e15 100644 --- a/docker/images.json +++ b/docker/images.json @@ -103,5 +103,25 @@ "docker/test/integration/helper_container": { "name": "yandex/clickhouse-integration-helper", "dependent": [] + }, + "docker/test/integration/mysql_golang_client": { + "name": "yandex/clickhouse-mysql-golang-client", + "dependent": [] + }, + "docker/test/integration/mysql_java_client": { + "name": "yandex/clickhouse-mysql-java-client", + "dependent": [] + }, + "docker/test/integration/mysql_js_client": { + "name": "yandex/clickhouse-mysql-js-client", + "dependent": [] + }, + "docker/test/integration/mysql_php_client": { + "name": "yandex/clickhouse-mysql-php-client", + "dependent": [] + }, + "docker/test/integration/postgresql_java_client": { + "name": "yandex/clickhouse-postgresql-java-client", + "dependent": [] } } diff --git a/tests/integration/test_mysql_protocol/clients/golang/Dockerfile b/docker/test/integration/mysql_golang_client/Dockerfile similarity index 52% rename from tests/integration/test_mysql_protocol/clients/golang/Dockerfile rename to docker/test/integration/mysql_golang_client/Dockerfile index d169c274a8b..4380383d1fb 100644 --- a/tests/integration/test_mysql_protocol/clients/golang/Dockerfile +++ b/docker/test/integration/mysql_golang_client/Dockerfile @@ -1,3 +1,6 @@ +# docker build -t yandex/clickhouse-mysql-golang-client . +# MySQL golang client docker container + FROM golang:1.12.2 RUN go get "github.com/go-sql-driver/mysql" diff --git a/tests/integration/test_mysql_protocol/clients/golang/main.go b/docker/test/integration/mysql_golang_client/main.go similarity index 100% rename from tests/integration/test_mysql_protocol/clients/golang/main.go rename to docker/test/integration/mysql_golang_client/main.go diff --git a/tests/integration/test_mysql_protocol/clients/java/Dockerfile b/docker/test/integration/mysql_java_client/Dockerfile similarity index 84% rename from tests/integration/test_mysql_protocol/clients/java/Dockerfile rename to docker/test/integration/mysql_java_client/Dockerfile index 96713a68e66..fcb6a39f33b 100644 --- a/tests/integration/test_mysql_protocol/clients/java/Dockerfile +++ b/docker/test/integration/mysql_java_client/Dockerfile @@ -1,3 +1,6 @@ +# docker build -t yandex/clickhouse-mysql-java-client . +# MySQL Java client docker container + FROM ubuntu:18.04 RUN apt-get update && \ diff --git a/tests/integration/test_mysql_protocol/clients/java/Test.java b/docker/test/integration/mysql_java_client/Test.java similarity index 100% rename from tests/integration/test_mysql_protocol/clients/java/Test.java rename to docker/test/integration/mysql_java_client/Test.java diff --git a/docker/test/integration/mysql_js_client/Dockerfile b/docker/test/integration/mysql_js_client/Dockerfile new file mode 100644 index 00000000000..4f12de004ac --- /dev/null +++ b/docker/test/integration/mysql_js_client/Dockerfile @@ -0,0 +1,8 @@ +# docker build -t yandex/clickhouse-mysql-js-client . +# MySQL JavaScript client docker container + +FROM node:8 + +RUN npm install mysql + +COPY ./test.js test.js diff --git a/tests/integration/test_mysql_protocol/clients/mysqljs/test.js b/docker/test/integration/mysql_js_client/test.js similarity index 100% rename from tests/integration/test_mysql_protocol/clients/mysqljs/test.js rename to docker/test/integration/mysql_js_client/test.js diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/Dockerfile b/docker/test/integration/mysql_php_client/Dockerfile similarity index 65% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/Dockerfile rename to docker/test/integration/mysql_php_client/Dockerfile index 76125702076..e2ceb62f44f 100644 --- a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/Dockerfile +++ b/docker/test/integration/mysql_php_client/Dockerfile @@ -1,3 +1,6 @@ +# docker build -t yandex/clickhouse-mysql-php-client . +# MySQL PHP client docker container + FROM php:7.3-cli COPY ./client.crt client.crt diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/client.crt b/docker/test/integration/mysql_php_client/client.crt similarity index 100% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/client.crt rename to docker/test/integration/mysql_php_client/client.crt diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/client.key b/docker/test/integration/mysql_php_client/client.key similarity index 100% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/client.key rename to docker/test/integration/mysql_php_client/client.key diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/test.php b/docker/test/integration/mysql_php_client/test.php similarity index 100% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/test.php rename to docker/test/integration/mysql_php_client/test.php diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/test_ssl.php b/docker/test/integration/mysql_php_client/test_ssl.php similarity index 100% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/test_ssl.php rename to docker/test/integration/mysql_php_client/test_ssl.php diff --git a/tests/integration/test_postgresql_protocol/clients/java/Dockerfile b/docker/test/integration/postgresql_java_client/Dockerfile similarity index 82% rename from tests/integration/test_postgresql_protocol/clients/java/Dockerfile rename to docker/test/integration/postgresql_java_client/Dockerfile index f08470ee805..eab236c9590 100644 --- a/tests/integration/test_postgresql_protocol/clients/java/Dockerfile +++ b/docker/test/integration/postgresql_java_client/Dockerfile @@ -1,3 +1,6 @@ +# docker build -t yandex/clickhouse-postgresql-java-client . +# PostgreSQL Java client docker container + FROM ubuntu:18.04 RUN apt-get update && \ diff --git a/tests/integration/test_postgresql_protocol/clients/java/Test.java b/docker/test/integration/postgresql_java_client/Test.java similarity index 100% rename from tests/integration/test_postgresql_protocol/clients/java/Test.java rename to docker/test/integration/postgresql_java_client/Test.java diff --git a/tests/integration/test_mysql_protocol/clients/mysql/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_client.yml similarity index 100% rename from tests/integration/test_mysql_protocol/clients/mysql/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_client.yml diff --git a/tests/integration/test_mysql_protocol/clients/golang/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml similarity index 53% rename from tests/integration/test_mysql_protocol/clients/golang/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml index 4fe6fdaeecd..b172cbcb2c6 100644 --- a/tests/integration/test_mysql_protocol/clients/golang/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml @@ -1,8 +1,6 @@ version: '2.3' services: golang1: - build: - context: ./ - network: host + image: yandex/clickhouse-mysql-golang-client:${DOCKER_MYSQL_GOLANG_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/tests/integration/test_mysql_protocol/clients/java/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml similarity index 54% rename from tests/integration/test_mysql_protocol/clients/java/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml index 522f404cde6..be1b3ad3f72 100644 --- a/tests/integration/test_mysql_protocol/clients/java/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml @@ -1,8 +1,6 @@ version: '2.3' services: java1: - build: - context: ./ - network: host + image: yandex/clickhouse-mysql-java-client:${DOCKER_MYSQL_JAVA_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml similarity index 56% rename from tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml index ebb73bc611a..83954229111 100644 --- a/tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml @@ -1,8 +1,6 @@ version: '2.3' services: mysqljs1: - build: - context: ./ - network: host + image: yandex/clickhouse-mysql-js-client:${DOCKER_MYSQL_JS_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml similarity index 54% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml index c197944f375..e61cb193b0e 100644 --- a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml @@ -1,7 +1,6 @@ version: '2.3' services: php1: - build: - context: ./ + image: yandex/clickhouse-mysql-php-client:${DOCKER_MYSQL_PHP_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/tests/integration/test_postgresql_protocol/clients/psql/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_postgesql.yml similarity index 100% rename from tests/integration/test_postgresql_protocol/clients/psql/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_postgesql.yml diff --git a/tests/integration/test_postgresql_protocol/clients/java/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml similarity index 51% rename from tests/integration/test_postgresql_protocol/clients/java/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml index 7094c8b2359..ef18d1edd7b 100644 --- a/tests/integration/test_postgresql_protocol/clients/java/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml @@ -1,8 +1,6 @@ version: '2.2' services: java: - build: - context: ./ - network: host + image: yandex/clickhouse-postgresql-java-client:${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 8b0682396f8..c38260279ed 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -22,5 +22,11 @@ export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge +export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} +export DOCKER_MYSQL_JAVA_CLIENT_TAG=${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest} +export DOCKER_MYSQL_JS_CLIENT_TAG=${DOCKER_MYSQL_JS_CLIENT_TAG:=latest} +export DOCKER_MYSQL_PHP_CLIENT_TAG=${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} +export DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} + cd /ClickHouse/tests/integration exec "$@" diff --git a/tests/integration/runner b/tests/integration/runner index 71993e47ff7..e5d6eabe794 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -139,6 +139,24 @@ if __name__ == "__main__": if not args.disable_net_host: net = "--net=host" + env_tags = "" + + if args.docker_compose_images_tags is not None: + for img_tag in args.docker_compose_images_tags: + [image, tag] = img_tag.split(":") + if image == "yandex/clickhouse-mysql-golang-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-java-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-js-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-php-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-postgresql-java-client": + env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) + else: + raise Exception("Unknown image {}".format(image)) + # create named volume which will be used inside to store images and other docker related files, # to avoid redownloading it every time # @@ -152,13 +170,14 @@ if __name__ == "__main__": cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ - --volume={name}_volume:/var/lib/docker -e PYTEST_OPTS='{opts}' {img} {command}".format( + --volume={name}_volume:/var/lib/docker {env_tags} -e PYTEST_OPTS='{opts}' {img} {command}".format( net=net, tty=tty, bin=args.binary, bridge_bin=args.bridge_binary, base_cfg=args.base_configs_dir, cases_dir=args.cases_dir, + env_tags=env_tags, opts=' '.join(args.pytest_args), img=DIND_INTEGRATION_TESTS_IMAGE_NAME + ":" + args.docker_image_version, name=CONTAINER_NAME, diff --git a/tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile b/tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile deleted file mode 100644 index 5381915efba..00000000000 --- a/tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile +++ /dev/null @@ -1,5 +0,0 @@ -FROM node:8 - -RUN npm install mysql - -COPY ./test.js test.js diff --git a/tests/integration/test_mysql_protocol/clients/golang/0.reference b/tests/integration/test_mysql_protocol/golang.reference similarity index 100% rename from tests/integration/test_mysql_protocol/clients/golang/0.reference rename to tests/integration/test_mysql_protocol/golang.reference diff --git a/tests/integration/test_mysql_protocol/clients/java/0.reference b/tests/integration/test_mysql_protocol/java.reference similarity index 100% rename from tests/integration/test_mysql_protocol/clients/java/0.reference rename to tests/integration/test_mysql_protocol/java.reference diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 4640c7b6b90..a31961dbd16 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -11,10 +11,11 @@ import pymysql.connections from docker.models.containers import Container -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, get_docker_compose_path SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DOCKER_COMPOSE_PATH = get_docker_compose_path() config_dir = os.path.join(SCRIPT_DIR, './configs') cluster = ClickHouseCluster(__file__) @@ -34,7 +35,7 @@ def server_address(): @pytest.fixture(scope='module') def mysql_client(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'mysql', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_mysql1_1') @@ -60,28 +61,28 @@ def mysql_server(mysql_client): @pytest.fixture(scope='module') def golang_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'golang', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_golang1_1') @pytest.fixture(scope='module') def php_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'php-mysqlnd', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_php1_1') @pytest.fixture(scope='module') def nodejs_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'mysqljs', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_mysqljs1_1') @pytest.fixture(scope='module') def java_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'java', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_java1_1') @@ -328,7 +329,7 @@ def test_python_client(server_address): def test_golang_client(server_address, golang_container): # type: (str, Container) -> None - with open(os.path.join(SCRIPT_DIR, 'clients', 'golang', '0.reference')) as fp: + with open(os.path.join(SCRIPT_DIR,'golang.reference')) as fp: reference = fp.read() code, (stdout, stderr) = golang_container.exec_run('./main --host {host} --port {port} --user default --password 123 --database ' @@ -385,7 +386,7 @@ def test_mysqljs_client(server_address, nodejs_container): def test_java_client(server_address, java_container): # type: (str, Container) -> None - with open(os.path.join(SCRIPT_DIR, 'clients', 'java', '0.reference')) as fp: + with open(os.path.join(SCRIPT_DIR, 'java.reference')) as fp: reference = fp.read() # database not exists exception. diff --git a/tests/integration/test_postgresql_protocol/clients/java/0.reference b/tests/integration/test_postgresql_protocol/java.reference similarity index 100% rename from tests/integration/test_postgresql_protocol/clients/java/0.reference rename to tests/integration/test_postgresql_protocol/java.reference diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index d9e2dfe3228..527c652229e 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -14,12 +14,13 @@ import subprocess import time import uuid -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, get_docker_compose_path psycopg2.extras.register_uuid() SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) config_dir = os.path.join(SCRIPT_DIR, './configs') +DOCKER_COMPOSE_PATH = get_docker_compose_path() cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', config_dir=config_dir, env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) @@ -38,7 +39,7 @@ def server_address(): @pytest.fixture(scope='module') def psql_client(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'psql', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_psql_1') @@ -61,7 +62,7 @@ def psql_server(psql_client): @pytest.fixture(scope='module') def java_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'java', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql_java_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_java_1') @@ -132,7 +133,7 @@ def test_python_client(server_address): def test_java_client(server_address, java_container): - with open(os.path.join(SCRIPT_DIR, 'clients', 'java', '0.reference')) as fp: + with open(os.path.join(SCRIPT_DIR, 'java.reference')) as fp: reference = fp.read() # database not exists exception. From fad9ef3671dd56f7646ed29012f57c37c7008b64 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <akuzm@yandex-team.ru> Date: Wed, 12 Aug 2020 19:00:28 +0300 Subject: [PATCH 293/374] Fix the 01428_hash_set_nan_key test not to rely on backwards-incompatible changes in `modulo` made in https://github.com/ClickHouse/ClickHouse/pull/12585 --- tests/queries/0_stateless/01428_hash_set_nan_key.sql | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01428_hash_set_nan_key.sql b/tests/queries/0_stateless/01428_hash_set_nan_key.sql index f9875f9ed99..837cd56a401 100644 --- a/tests/queries/0_stateless/01428_hash_set_nan_key.sql +++ b/tests/queries/0_stateless/01428_hash_set_nan_key.sql @@ -1,10 +1,10 @@ SELECT uniqExact(nan) FROM numbers(1000); -SELECT uniqExact(number % inf) FROM numbers(1000); -SELECT sumDistinct(number % inf) FROM numbers(1000); -SELECT DISTINCT number % inf FROM numbers(1000); +SELECT uniqExact(number + nan) FROM numbers(1000); +SELECT sumDistinct(number + nan) FROM numbers(1000); +SELECT DISTINCT number + nan FROM numbers(1000); SELECT topKWeightedMerge(1)(initializeAggregation('topKWeightedState(1)', nan, arrayJoin(range(10)))); -select number % inf k from numbers(256) group by k order by k; +select number + nan k from numbers(256) group by k; -SELECT uniqExact(reinterpretAsFloat64(reinterpretAsFixedString(reinterpretAsUInt64(reinterpretAsFixedString(nan)) + number))) n FROM numbers(10) order by n; +SELECT uniqExact(reinterpretAsFloat64(reinterpretAsFixedString(reinterpretAsUInt64(reinterpretAsFixedString(nan)) + number))) FROM numbers(10); From 74832999cc574ee1da83941ab61da4501a81afb9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed, 12 Aug 2020 21:23:18 +0300 Subject: [PATCH 294/374] Update a story --- benchmark/monetdb/usability.md | 860 +++++++++++++++++---------------- 1 file changed, 434 insertions(+), 426 deletions(-) diff --git a/benchmark/monetdb/usability.md b/benchmark/monetdb/usability.md index 1f39b3a434c..20eaf37f9e8 100644 --- a/benchmark/monetdb/usability.md +++ b/benchmark/monetdb/usability.md @@ -231,111 +231,111 @@ Maybe it's a trick to optimize NULLs? Let's just cheat and add one to all the most negative numbers while exporting dataset from ClickHouse... ``` -SELECT - toInt64(WatchID) = -9223372036854775808 ? -9223372036854775807 : toInt64(WatchID), - toInt8(JavaEnable) = -128 ? -127 : toInt8(JavaEnable), - toValidUTF8(toString(Title)), - toInt16(GoodEvent) = -32768 ? -32767 : toInt16(GoodEvent), - EventTime, - EventDate, - toInt32(CounterID) = -2147483648 ? -2147483647 : toInt32(CounterID), - toInt32(ClientIP) = -2147483648 ? -2147483647 : toInt32(ClientIP), - toInt32(RegionID) = -2147483648 ? -2147483647 : toInt32(RegionID), - toInt64(UserID) = -9223372036854775808 ? -9223372036854775807 : toInt64(UserID), - toInt8(CounterClass) = -128 ? -127 : toInt8(CounterClass), - toInt8(OS) = -128 ? -127 : toInt8(OS), - toInt8(UserAgent) = -128 ? -127 : toInt8(UserAgent), - toValidUTF8(toString(URL)), - toValidUTF8(toString(Referer)), - toInt8(Refresh) = -128 ? -127 : toInt8(Refresh), - toInt16(RefererCategoryID) = -32768 ? -32767 : toInt16(RefererCategoryID), - toInt32(RefererRegionID) = -2147483648 ? -2147483647 : toInt32(RefererRegionID), - toInt16(URLCategoryID) = -32768 ? -32767 : toInt16(URLCategoryID), - toInt32(URLRegionID) = -2147483648 ? -2147483647 : toInt32(URLRegionID), - toInt16(ResolutionWidth) = -32768 ? -32767 : toInt16(ResolutionWidth), - toInt16(ResolutionHeight) = -32768 ? -32767 : toInt16(ResolutionHeight), - toInt8(ResolutionDepth) = -128 ? -127 : toInt8(ResolutionDepth), - toInt8(FlashMajor) = -128 ? -127 : toInt8(FlashMajor), - toInt8(FlashMinor) = -128 ? -127 : toInt8(FlashMinor), - toValidUTF8(toString(FlashMinor2)), - toInt8(NetMajor) = -128 ? -127 : toInt8(NetMajor), - toInt8(NetMinor) = -128 ? -127 : toInt8(NetMinor), - toInt16(UserAgentMajor) = -32768 ? -32767 : toInt16(UserAgentMajor), - toValidUTF8(toString(UserAgentMinor)), - toInt8(CookieEnable) = -128 ? -127 : toInt8(CookieEnable), - toInt8(JavascriptEnable) = -128 ? -127 : toInt8(JavascriptEnable), - toInt8(IsMobile) = -128 ? -127 : toInt8(IsMobile), - toInt8(MobilePhone) = -128 ? -127 : toInt8(MobilePhone), - toValidUTF8(toString(MobilePhoneModel)), - toValidUTF8(toString(Params)), - toInt32(IPNetworkID) = -2147483648 ? -2147483647 : toInt32(IPNetworkID), - toInt8(TraficSourceID) = -128 ? -127 : toInt8(TraficSourceID), - toInt16(SearchEngineID) = -32768 ? -32767 : toInt16(SearchEngineID), - toValidUTF8(toString(SearchPhrase)), - toInt8(AdvEngineID) = -128 ? -127 : toInt8(AdvEngineID), - toInt8(IsArtifical) = -128 ? -127 : toInt8(IsArtifical), - toInt16(WindowClientWidth) = -32768 ? -32767 : toInt16(WindowClientWidth), - toInt16(WindowClientHeight) = -32768 ? -32767 : toInt16(WindowClientHeight), - toInt16(ClientTimeZone) = -32768 ? -32767 : toInt16(ClientTimeZone), - ClientEventTime, - toInt8(SilverlightVersion1) = -128 ? -127 : toInt8(SilverlightVersion1), - toInt8(SilverlightVersion2) = -128 ? -127 : toInt8(SilverlightVersion2), - toInt32(SilverlightVersion3) = -2147483648 ? -2147483647 : toInt32(SilverlightVersion3), - toInt16(SilverlightVersion4) = -32768 ? -32767 : toInt16(SilverlightVersion4), - toValidUTF8(toString(PageCharset)), - toInt32(CodeVersion) = -2147483648 ? -2147483647 : toInt32(CodeVersion), - toInt8(IsLink) = -128 ? -127 : toInt8(IsLink), - toInt8(IsDownload) = -128 ? -127 : toInt8(IsDownload), - toInt8(IsNotBounce) = -128 ? -127 : toInt8(IsNotBounce), - toInt64(FUniqID) = -9223372036854775808 ? -9223372036854775807 : toInt64(FUniqID), - toValidUTF8(toString(OriginalURL)), - toInt32(HID) = -2147483648 ? -2147483647 : toInt32(HID), - toInt8(IsOldCounter) = -128 ? -127 : toInt8(IsOldCounter), - toInt8(IsEvent) = -128 ? -127 : toInt8(IsEvent), - toInt8(IsParameter) = -128 ? -127 : toInt8(IsParameter), - toInt8(DontCountHits) = -128 ? -127 : toInt8(DontCountHits), - toInt8(WithHash) = -128 ? -127 : toInt8(WithHash), - toValidUTF8(toString(HitColor)), - LocalEventTime, - toInt8(Age) = -128 ? -127 : toInt8(Age), - toInt8(Sex) = -128 ? -127 : toInt8(Sex), - toInt8(Income) = -128 ? -127 : toInt8(Income), - toInt16(Interests) = -32768 ? -32767 : toInt16(Interests), - toInt8(Robotness) = -128 ? -127 : toInt8(Robotness), - toInt32(RemoteIP) = -2147483648 ? -2147483647 : toInt32(RemoteIP), - toInt32(WindowName) = -2147483648 ? -2147483647 : toInt32(WindowName), - toInt32(OpenerName) = -2147483648 ? -2147483647 : toInt32(OpenerName), - toInt16(HistoryLength) = -32768 ? -32767 : toInt16(HistoryLength), - toValidUTF8(toString(BrowserLanguage)), - toValidUTF8(toString(BrowserCountry)), - toValidUTF8(toString(SocialNetwork)), - toValidUTF8(toString(SocialAction)), - toInt16(HTTPError) = -32768 ? -32767 : toInt16(HTTPError), - toInt32(SendTiming) = -2147483648 ? -2147483647 : toInt32(SendTiming), - toInt32(DNSTiming) = -2147483648 ? -2147483647 : toInt32(DNSTiming), - toInt32(ConnectTiming) = -2147483648 ? -2147483647 : toInt32(ConnectTiming), - toInt32(ResponseStartTiming) = -2147483648 ? -2147483647 : toInt32(ResponseStartTiming), - toInt32(ResponseEndTiming) = -2147483648 ? -2147483647 : toInt32(ResponseEndTiming), - toInt32(FetchTiming) = -2147483648 ? -2147483647 : toInt32(FetchTiming), - toInt8(SocialSourceNetworkID) = -128 ? -127 : toInt8(SocialSourceNetworkID), - toValidUTF8(toString(SocialSourcePage)), - toInt64(ParamPrice) = -9223372036854775808 ? -9223372036854775807 : toInt64(ParamPrice), - toValidUTF8(toString(ParamOrderID)), - toValidUTF8(toString(ParamCurrency)), - toInt16(ParamCurrencyID) = -32768 ? -32767 : toInt16(ParamCurrencyID), - toValidUTF8(toString(OpenstatServiceName)), - toValidUTF8(toString(OpenstatCampaignID)), - toValidUTF8(toString(OpenstatAdID)), - toValidUTF8(toString(OpenstatSourceID)), - toValidUTF8(toString(UTMSource)), - toValidUTF8(toString(UTMMedium)), - toValidUTF8(toString(UTMCampaign)), - toValidUTF8(toString(UTMContent)), - toValidUTF8(toString(UTMTerm)), - toValidUTF8(toString(FromTag)), - toInt8(HasGCLID) = -128 ? -127 : toInt8(HasGCLID), - toInt64(RefererHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(RefererHash), - toInt64(URLHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(URLHash), +SELECT + toInt64(WatchID) = -9223372036854775808 ? -9223372036854775807 : toInt64(WatchID), + toInt8(JavaEnable) = -128 ? -127 : toInt8(JavaEnable), + toValidUTF8(toString(Title)), + toInt16(GoodEvent) = -32768 ? -32767 : toInt16(GoodEvent), + EventTime, + EventDate, + toInt32(CounterID) = -2147483648 ? -2147483647 : toInt32(CounterID), + toInt32(ClientIP) = -2147483648 ? -2147483647 : toInt32(ClientIP), + toInt32(RegionID) = -2147483648 ? -2147483647 : toInt32(RegionID), + toInt64(UserID) = -9223372036854775808 ? -9223372036854775807 : toInt64(UserID), + toInt8(CounterClass) = -128 ? -127 : toInt8(CounterClass), + toInt8(OS) = -128 ? -127 : toInt8(OS), + toInt8(UserAgent) = -128 ? -127 : toInt8(UserAgent), + toValidUTF8(toString(URL)), + toValidUTF8(toString(Referer)), + toInt8(Refresh) = -128 ? -127 : toInt8(Refresh), + toInt16(RefererCategoryID) = -32768 ? -32767 : toInt16(RefererCategoryID), + toInt32(RefererRegionID) = -2147483648 ? -2147483647 : toInt32(RefererRegionID), + toInt16(URLCategoryID) = -32768 ? -32767 : toInt16(URLCategoryID), + toInt32(URLRegionID) = -2147483648 ? -2147483647 : toInt32(URLRegionID), + toInt16(ResolutionWidth) = -32768 ? -32767 : toInt16(ResolutionWidth), + toInt16(ResolutionHeight) = -32768 ? -32767 : toInt16(ResolutionHeight), + toInt8(ResolutionDepth) = -128 ? -127 : toInt8(ResolutionDepth), + toInt8(FlashMajor) = -128 ? -127 : toInt8(FlashMajor), + toInt8(FlashMinor) = -128 ? -127 : toInt8(FlashMinor), + toValidUTF8(toString(FlashMinor2)), + toInt8(NetMajor) = -128 ? -127 : toInt8(NetMajor), + toInt8(NetMinor) = -128 ? -127 : toInt8(NetMinor), + toInt16(UserAgentMajor) = -32768 ? -32767 : toInt16(UserAgentMajor), + toValidUTF8(toString(UserAgentMinor)), + toInt8(CookieEnable) = -128 ? -127 : toInt8(CookieEnable), + toInt8(JavascriptEnable) = -128 ? -127 : toInt8(JavascriptEnable), + toInt8(IsMobile) = -128 ? -127 : toInt8(IsMobile), + toInt8(MobilePhone) = -128 ? -127 : toInt8(MobilePhone), + toValidUTF8(toString(MobilePhoneModel)), + toValidUTF8(toString(Params)), + toInt32(IPNetworkID) = -2147483648 ? -2147483647 : toInt32(IPNetworkID), + toInt8(TraficSourceID) = -128 ? -127 : toInt8(TraficSourceID), + toInt16(SearchEngineID) = -32768 ? -32767 : toInt16(SearchEngineID), + toValidUTF8(toString(SearchPhrase)), + toInt8(AdvEngineID) = -128 ? -127 : toInt8(AdvEngineID), + toInt8(IsArtifical) = -128 ? -127 : toInt8(IsArtifical), + toInt16(WindowClientWidth) = -32768 ? -32767 : toInt16(WindowClientWidth), + toInt16(WindowClientHeight) = -32768 ? -32767 : toInt16(WindowClientHeight), + toInt16(ClientTimeZone) = -32768 ? -32767 : toInt16(ClientTimeZone), + ClientEventTime, + toInt8(SilverlightVersion1) = -128 ? -127 : toInt8(SilverlightVersion1), + toInt8(SilverlightVersion2) = -128 ? -127 : toInt8(SilverlightVersion2), + toInt32(SilverlightVersion3) = -2147483648 ? -2147483647 : toInt32(SilverlightVersion3), + toInt16(SilverlightVersion4) = -32768 ? -32767 : toInt16(SilverlightVersion4), + toValidUTF8(toString(PageCharset)), + toInt32(CodeVersion) = -2147483648 ? -2147483647 : toInt32(CodeVersion), + toInt8(IsLink) = -128 ? -127 : toInt8(IsLink), + toInt8(IsDownload) = -128 ? -127 : toInt8(IsDownload), + toInt8(IsNotBounce) = -128 ? -127 : toInt8(IsNotBounce), + toInt64(FUniqID) = -9223372036854775808 ? -9223372036854775807 : toInt64(FUniqID), + toValidUTF8(toString(OriginalURL)), + toInt32(HID) = -2147483648 ? -2147483647 : toInt32(HID), + toInt8(IsOldCounter) = -128 ? -127 : toInt8(IsOldCounter), + toInt8(IsEvent) = -128 ? -127 : toInt8(IsEvent), + toInt8(IsParameter) = -128 ? -127 : toInt8(IsParameter), + toInt8(DontCountHits) = -128 ? -127 : toInt8(DontCountHits), + toInt8(WithHash) = -128 ? -127 : toInt8(WithHash), + toValidUTF8(toString(HitColor)), + LocalEventTime, + toInt8(Age) = -128 ? -127 : toInt8(Age), + toInt8(Sex) = -128 ? -127 : toInt8(Sex), + toInt8(Income) = -128 ? -127 : toInt8(Income), + toInt16(Interests) = -32768 ? -32767 : toInt16(Interests), + toInt8(Robotness) = -128 ? -127 : toInt8(Robotness), + toInt32(RemoteIP) = -2147483648 ? -2147483647 : toInt32(RemoteIP), + toInt32(WindowName) = -2147483648 ? -2147483647 : toInt32(WindowName), + toInt32(OpenerName) = -2147483648 ? -2147483647 : toInt32(OpenerName), + toInt16(HistoryLength) = -32768 ? -32767 : toInt16(HistoryLength), + toValidUTF8(toString(BrowserLanguage)), + toValidUTF8(toString(BrowserCountry)), + toValidUTF8(toString(SocialNetwork)), + toValidUTF8(toString(SocialAction)), + toInt16(HTTPError) = -32768 ? -32767 : toInt16(HTTPError), + toInt32(SendTiming) = -2147483648 ? -2147483647 : toInt32(SendTiming), + toInt32(DNSTiming) = -2147483648 ? -2147483647 : toInt32(DNSTiming), + toInt32(ConnectTiming) = -2147483648 ? -2147483647 : toInt32(ConnectTiming), + toInt32(ResponseStartTiming) = -2147483648 ? -2147483647 : toInt32(ResponseStartTiming), + toInt32(ResponseEndTiming) = -2147483648 ? -2147483647 : toInt32(ResponseEndTiming), + toInt32(FetchTiming) = -2147483648 ? -2147483647 : toInt32(FetchTiming), + toInt8(SocialSourceNetworkID) = -128 ? -127 : toInt8(SocialSourceNetworkID), + toValidUTF8(toString(SocialSourcePage)), + toInt64(ParamPrice) = -9223372036854775808 ? -9223372036854775807 : toInt64(ParamPrice), + toValidUTF8(toString(ParamOrderID)), + toValidUTF8(toString(ParamCurrency)), + toInt16(ParamCurrencyID) = -32768 ? -32767 : toInt16(ParamCurrencyID), + toValidUTF8(toString(OpenstatServiceName)), + toValidUTF8(toString(OpenstatCampaignID)), + toValidUTF8(toString(OpenstatAdID)), + toValidUTF8(toString(OpenstatSourceID)), + toValidUTF8(toString(UTMSource)), + toValidUTF8(toString(UTMMedium)), + toValidUTF8(toString(UTMCampaign)), + toValidUTF8(toString(UTMContent)), + toValidUTF8(toString(UTMTerm)), + toValidUTF8(toString(FromTag)), + toInt8(HasGCLID) = -128 ? -127 : toInt8(HasGCLID), + toInt64(RefererHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(RefererHash), + toInt64(URLHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(URLHash), toInt32(CLID) = -2147483648 ? -2147483647 : toInt32(CLID) FROM hits_100m_obfuscated INTO OUTFILE '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' @@ -366,111 +366,111 @@ Ok, then it's TSV, not CSV. Let's create TSV dump... ``` -SELECT - toInt64(WatchID) = -9223372036854775808 ? -9223372036854775807 : toInt64(WatchID), - toInt8(JavaEnable) = -128 ? -127 : toInt8(JavaEnable), - toValidUTF8(toString(Title)), - toInt16(GoodEvent) = -32768 ? -32767 : toInt16(GoodEvent), - EventTime, - EventDate, - toInt32(CounterID) = -2147483648 ? -2147483647 : toInt32(CounterID), - toInt32(ClientIP) = -2147483648 ? -2147483647 : toInt32(ClientIP), - toInt32(RegionID) = -2147483648 ? -2147483647 : toInt32(RegionID), - toInt64(UserID) = -9223372036854775808 ? -9223372036854775807 : toInt64(UserID), - toInt8(CounterClass) = -128 ? -127 : toInt8(CounterClass), - toInt8(OS) = -128 ? -127 : toInt8(OS), - toInt8(UserAgent) = -128 ? -127 : toInt8(UserAgent), - toValidUTF8(toString(URL)), - toValidUTF8(toString(Referer)), - toInt8(Refresh) = -128 ? -127 : toInt8(Refresh), - toInt16(RefererCategoryID) = -32768 ? -32767 : toInt16(RefererCategoryID), - toInt32(RefererRegionID) = -2147483648 ? -2147483647 : toInt32(RefererRegionID), - toInt16(URLCategoryID) = -32768 ? -32767 : toInt16(URLCategoryID), - toInt32(URLRegionID) = -2147483648 ? -2147483647 : toInt32(URLRegionID), - toInt16(ResolutionWidth) = -32768 ? -32767 : toInt16(ResolutionWidth), - toInt16(ResolutionHeight) = -32768 ? -32767 : toInt16(ResolutionHeight), - toInt8(ResolutionDepth) = -128 ? -127 : toInt8(ResolutionDepth), - toInt8(FlashMajor) = -128 ? -127 : toInt8(FlashMajor), - toInt8(FlashMinor) = -128 ? -127 : toInt8(FlashMinor), - toValidUTF8(toString(FlashMinor2)), - toInt8(NetMajor) = -128 ? -127 : toInt8(NetMajor), - toInt8(NetMinor) = -128 ? -127 : toInt8(NetMinor), - toInt16(UserAgentMajor) = -32768 ? -32767 : toInt16(UserAgentMajor), - toValidUTF8(toString(UserAgentMinor)), - toInt8(CookieEnable) = -128 ? -127 : toInt8(CookieEnable), - toInt8(JavascriptEnable) = -128 ? -127 : toInt8(JavascriptEnable), - toInt8(IsMobile) = -128 ? -127 : toInt8(IsMobile), - toInt8(MobilePhone) = -128 ? -127 : toInt8(MobilePhone), - toValidUTF8(toString(MobilePhoneModel)), - toValidUTF8(toString(Params)), - toInt32(IPNetworkID) = -2147483648 ? -2147483647 : toInt32(IPNetworkID), - toInt8(TraficSourceID) = -128 ? -127 : toInt8(TraficSourceID), - toInt16(SearchEngineID) = -32768 ? -32767 : toInt16(SearchEngineID), - toValidUTF8(toString(SearchPhrase)), - toInt8(AdvEngineID) = -128 ? -127 : toInt8(AdvEngineID), - toInt8(IsArtifical) = -128 ? -127 : toInt8(IsArtifical), - toInt16(WindowClientWidth) = -32768 ? -32767 : toInt16(WindowClientWidth), - toInt16(WindowClientHeight) = -32768 ? -32767 : toInt16(WindowClientHeight), - toInt16(ClientTimeZone) = -32768 ? -32767 : toInt16(ClientTimeZone), - ClientEventTime, - toInt8(SilverlightVersion1) = -128 ? -127 : toInt8(SilverlightVersion1), - toInt8(SilverlightVersion2) = -128 ? -127 : toInt8(SilverlightVersion2), - toInt32(SilverlightVersion3) = -2147483648 ? -2147483647 : toInt32(SilverlightVersion3), - toInt16(SilverlightVersion4) = -32768 ? -32767 : toInt16(SilverlightVersion4), - toValidUTF8(toString(PageCharset)), - toInt32(CodeVersion) = -2147483648 ? -2147483647 : toInt32(CodeVersion), - toInt8(IsLink) = -128 ? -127 : toInt8(IsLink), - toInt8(IsDownload) = -128 ? -127 : toInt8(IsDownload), - toInt8(IsNotBounce) = -128 ? -127 : toInt8(IsNotBounce), - toInt64(FUniqID) = -9223372036854775808 ? -9223372036854775807 : toInt64(FUniqID), - toValidUTF8(toString(OriginalURL)), - toInt32(HID) = -2147483648 ? -2147483647 : toInt32(HID), - toInt8(IsOldCounter) = -128 ? -127 : toInt8(IsOldCounter), - toInt8(IsEvent) = -128 ? -127 : toInt8(IsEvent), - toInt8(IsParameter) = -128 ? -127 : toInt8(IsParameter), - toInt8(DontCountHits) = -128 ? -127 : toInt8(DontCountHits), - toInt8(WithHash) = -128 ? -127 : toInt8(WithHash), - toValidUTF8(toString(HitColor)), - LocalEventTime, - toInt8(Age) = -128 ? -127 : toInt8(Age), - toInt8(Sex) = -128 ? -127 : toInt8(Sex), - toInt8(Income) = -128 ? -127 : toInt8(Income), - toInt16(Interests) = -32768 ? -32767 : toInt16(Interests), - toInt8(Robotness) = -128 ? -127 : toInt8(Robotness), - toInt32(RemoteIP) = -2147483648 ? -2147483647 : toInt32(RemoteIP), - toInt32(WindowName) = -2147483648 ? -2147483647 : toInt32(WindowName), - toInt32(OpenerName) = -2147483648 ? -2147483647 : toInt32(OpenerName), - toInt16(HistoryLength) = -32768 ? -32767 : toInt16(HistoryLength), - toValidUTF8(toString(BrowserLanguage)), - toValidUTF8(toString(BrowserCountry)), - toValidUTF8(toString(SocialNetwork)), - toValidUTF8(toString(SocialAction)), - toInt16(HTTPError) = -32768 ? -32767 : toInt16(HTTPError), - toInt32(SendTiming) = -2147483648 ? -2147483647 : toInt32(SendTiming), - toInt32(DNSTiming) = -2147483648 ? -2147483647 : toInt32(DNSTiming), - toInt32(ConnectTiming) = -2147483648 ? -2147483647 : toInt32(ConnectTiming), - toInt32(ResponseStartTiming) = -2147483648 ? -2147483647 : toInt32(ResponseStartTiming), - toInt32(ResponseEndTiming) = -2147483648 ? -2147483647 : toInt32(ResponseEndTiming), - toInt32(FetchTiming) = -2147483648 ? -2147483647 : toInt32(FetchTiming), - toInt8(SocialSourceNetworkID) = -128 ? -127 : toInt8(SocialSourceNetworkID), - toValidUTF8(toString(SocialSourcePage)), - toInt64(ParamPrice) = -9223372036854775808 ? -9223372036854775807 : toInt64(ParamPrice), - toValidUTF8(toString(ParamOrderID)), - toValidUTF8(toString(ParamCurrency)), - toInt16(ParamCurrencyID) = -32768 ? -32767 : toInt16(ParamCurrencyID), - toValidUTF8(toString(OpenstatServiceName)), - toValidUTF8(toString(OpenstatCampaignID)), - toValidUTF8(toString(OpenstatAdID)), - toValidUTF8(toString(OpenstatSourceID)), - toValidUTF8(toString(UTMSource)), - toValidUTF8(toString(UTMMedium)), - toValidUTF8(toString(UTMCampaign)), - toValidUTF8(toString(UTMContent)), - toValidUTF8(toString(UTMTerm)), - toValidUTF8(toString(FromTag)), - toInt8(HasGCLID) = -128 ? -127 : toInt8(HasGCLID), - toInt64(RefererHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(RefererHash), - toInt64(URLHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(URLHash), +SELECT + toInt64(WatchID) = -9223372036854775808 ? -9223372036854775807 : toInt64(WatchID), + toInt8(JavaEnable) = -128 ? -127 : toInt8(JavaEnable), + toValidUTF8(toString(Title)), + toInt16(GoodEvent) = -32768 ? -32767 : toInt16(GoodEvent), + EventTime, + EventDate, + toInt32(CounterID) = -2147483648 ? -2147483647 : toInt32(CounterID), + toInt32(ClientIP) = -2147483648 ? -2147483647 : toInt32(ClientIP), + toInt32(RegionID) = -2147483648 ? -2147483647 : toInt32(RegionID), + toInt64(UserID) = -9223372036854775808 ? -9223372036854775807 : toInt64(UserID), + toInt8(CounterClass) = -128 ? -127 : toInt8(CounterClass), + toInt8(OS) = -128 ? -127 : toInt8(OS), + toInt8(UserAgent) = -128 ? -127 : toInt8(UserAgent), + toValidUTF8(toString(URL)), + toValidUTF8(toString(Referer)), + toInt8(Refresh) = -128 ? -127 : toInt8(Refresh), + toInt16(RefererCategoryID) = -32768 ? -32767 : toInt16(RefererCategoryID), + toInt32(RefererRegionID) = -2147483648 ? -2147483647 : toInt32(RefererRegionID), + toInt16(URLCategoryID) = -32768 ? -32767 : toInt16(URLCategoryID), + toInt32(URLRegionID) = -2147483648 ? -2147483647 : toInt32(URLRegionID), + toInt16(ResolutionWidth) = -32768 ? -32767 : toInt16(ResolutionWidth), + toInt16(ResolutionHeight) = -32768 ? -32767 : toInt16(ResolutionHeight), + toInt8(ResolutionDepth) = -128 ? -127 : toInt8(ResolutionDepth), + toInt8(FlashMajor) = -128 ? -127 : toInt8(FlashMajor), + toInt8(FlashMinor) = -128 ? -127 : toInt8(FlashMinor), + toValidUTF8(toString(FlashMinor2)), + toInt8(NetMajor) = -128 ? -127 : toInt8(NetMajor), + toInt8(NetMinor) = -128 ? -127 : toInt8(NetMinor), + toInt16(UserAgentMajor) = -32768 ? -32767 : toInt16(UserAgentMajor), + toValidUTF8(toString(UserAgentMinor)), + toInt8(CookieEnable) = -128 ? -127 : toInt8(CookieEnable), + toInt8(JavascriptEnable) = -128 ? -127 : toInt8(JavascriptEnable), + toInt8(IsMobile) = -128 ? -127 : toInt8(IsMobile), + toInt8(MobilePhone) = -128 ? -127 : toInt8(MobilePhone), + toValidUTF8(toString(MobilePhoneModel)), + toValidUTF8(toString(Params)), + toInt32(IPNetworkID) = -2147483648 ? -2147483647 : toInt32(IPNetworkID), + toInt8(TraficSourceID) = -128 ? -127 : toInt8(TraficSourceID), + toInt16(SearchEngineID) = -32768 ? -32767 : toInt16(SearchEngineID), + toValidUTF8(toString(SearchPhrase)), + toInt8(AdvEngineID) = -128 ? -127 : toInt8(AdvEngineID), + toInt8(IsArtifical) = -128 ? -127 : toInt8(IsArtifical), + toInt16(WindowClientWidth) = -32768 ? -32767 : toInt16(WindowClientWidth), + toInt16(WindowClientHeight) = -32768 ? -32767 : toInt16(WindowClientHeight), + toInt16(ClientTimeZone) = -32768 ? -32767 : toInt16(ClientTimeZone), + ClientEventTime, + toInt8(SilverlightVersion1) = -128 ? -127 : toInt8(SilverlightVersion1), + toInt8(SilverlightVersion2) = -128 ? -127 : toInt8(SilverlightVersion2), + toInt32(SilverlightVersion3) = -2147483648 ? -2147483647 : toInt32(SilverlightVersion3), + toInt16(SilverlightVersion4) = -32768 ? -32767 : toInt16(SilverlightVersion4), + toValidUTF8(toString(PageCharset)), + toInt32(CodeVersion) = -2147483648 ? -2147483647 : toInt32(CodeVersion), + toInt8(IsLink) = -128 ? -127 : toInt8(IsLink), + toInt8(IsDownload) = -128 ? -127 : toInt8(IsDownload), + toInt8(IsNotBounce) = -128 ? -127 : toInt8(IsNotBounce), + toInt64(FUniqID) = -9223372036854775808 ? -9223372036854775807 : toInt64(FUniqID), + toValidUTF8(toString(OriginalURL)), + toInt32(HID) = -2147483648 ? -2147483647 : toInt32(HID), + toInt8(IsOldCounter) = -128 ? -127 : toInt8(IsOldCounter), + toInt8(IsEvent) = -128 ? -127 : toInt8(IsEvent), + toInt8(IsParameter) = -128 ? -127 : toInt8(IsParameter), + toInt8(DontCountHits) = -128 ? -127 : toInt8(DontCountHits), + toInt8(WithHash) = -128 ? -127 : toInt8(WithHash), + toValidUTF8(toString(HitColor)), + LocalEventTime, + toInt8(Age) = -128 ? -127 : toInt8(Age), + toInt8(Sex) = -128 ? -127 : toInt8(Sex), + toInt8(Income) = -128 ? -127 : toInt8(Income), + toInt16(Interests) = -32768 ? -32767 : toInt16(Interests), + toInt8(Robotness) = -128 ? -127 : toInt8(Robotness), + toInt32(RemoteIP) = -2147483648 ? -2147483647 : toInt32(RemoteIP), + toInt32(WindowName) = -2147483648 ? -2147483647 : toInt32(WindowName), + toInt32(OpenerName) = -2147483648 ? -2147483647 : toInt32(OpenerName), + toInt16(HistoryLength) = -32768 ? -32767 : toInt16(HistoryLength), + toValidUTF8(toString(BrowserLanguage)), + toValidUTF8(toString(BrowserCountry)), + toValidUTF8(toString(SocialNetwork)), + toValidUTF8(toString(SocialAction)), + toInt16(HTTPError) = -32768 ? -32767 : toInt16(HTTPError), + toInt32(SendTiming) = -2147483648 ? -2147483647 : toInt32(SendTiming), + toInt32(DNSTiming) = -2147483648 ? -2147483647 : toInt32(DNSTiming), + toInt32(ConnectTiming) = -2147483648 ? -2147483647 : toInt32(ConnectTiming), + toInt32(ResponseStartTiming) = -2147483648 ? -2147483647 : toInt32(ResponseStartTiming), + toInt32(ResponseEndTiming) = -2147483648 ? -2147483647 : toInt32(ResponseEndTiming), + toInt32(FetchTiming) = -2147483648 ? -2147483647 : toInt32(FetchTiming), + toInt8(SocialSourceNetworkID) = -128 ? -127 : toInt8(SocialSourceNetworkID), + toValidUTF8(toString(SocialSourcePage)), + toInt64(ParamPrice) = -9223372036854775808 ? -9223372036854775807 : toInt64(ParamPrice), + toValidUTF8(toString(ParamOrderID)), + toValidUTF8(toString(ParamCurrency)), + toInt16(ParamCurrencyID) = -32768 ? -32767 : toInt16(ParamCurrencyID), + toValidUTF8(toString(OpenstatServiceName)), + toValidUTF8(toString(OpenstatCampaignID)), + toValidUTF8(toString(OpenstatAdID)), + toValidUTF8(toString(OpenstatSourceID)), + toValidUTF8(toString(UTMSource)), + toValidUTF8(toString(UTMMedium)), + toValidUTF8(toString(UTMCampaign)), + toValidUTF8(toString(UTMContent)), + toValidUTF8(toString(UTMTerm)), + toValidUTF8(toString(FromTag)), + toInt8(HasGCLID) = -128 ? -127 : toInt8(HasGCLID), + toInt64(RefererHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(RefererHash), + toInt64(URLHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(URLHash), toInt32(CLID) = -2147483648 ? -2147483647 : toInt32(CLID) FROM hits_100m_obfuscated INTO OUTFILE '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' @@ -485,7 +485,7 @@ MonetDB client lacks history. ``` sql>COPY 100000000 RECORDS INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' USING DELIMITERS '\t', '\n', ''; -Failed to import table 'hits', +Failed to import table 'hits', clk: 1.200 sec ``` @@ -503,7 +503,7 @@ Ok, it appeared to work... ``` sql>COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' USING DELIMITERS '\t', '\n', ''; -Failed to import table 'hits', +Failed to import table 'hits', clk: 2:31 min ``` @@ -548,111 +548,111 @@ BTW, the favicon of the MonetDB website makes an impression that the web page is Let's cheat again and replace all line feeds in strings to whitespaces and all double quotes to single quotes. ``` -SELECT - toInt64(WatchID) = -9223372036854775808 ? -9223372036854775807 : toInt64(WatchID), - toInt8(JavaEnable) = -128 ? -127 : toInt8(JavaEnable), - replaceAll(replaceAll(toValidUTF8(toString(Title)), '\n', ' '), '"', '\''), - toInt16(GoodEvent) = -32768 ? -32767 : toInt16(GoodEvent), - EventTime, - EventDate, - toInt32(CounterID) = -2147483648 ? -2147483647 : toInt32(CounterID), - toInt32(ClientIP) = -2147483648 ? -2147483647 : toInt32(ClientIP), - toInt32(RegionID) = -2147483648 ? -2147483647 : toInt32(RegionID), - toInt64(UserID) = -9223372036854775808 ? -9223372036854775807 : toInt64(UserID), - toInt8(CounterClass) = -128 ? -127 : toInt8(CounterClass), - toInt8(OS) = -128 ? -127 : toInt8(OS), - toInt8(UserAgent) = -128 ? -127 : toInt8(UserAgent), - replaceAll(replaceAll(toValidUTF8(toString(URL)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(Referer)), '\n', ' '), '"', '\''), - toInt8(Refresh) = -128 ? -127 : toInt8(Refresh), - toInt16(RefererCategoryID) = -32768 ? -32767 : toInt16(RefererCategoryID), - toInt32(RefererRegionID) = -2147483648 ? -2147483647 : toInt32(RefererRegionID), - toInt16(URLCategoryID) = -32768 ? -32767 : toInt16(URLCategoryID), - toInt32(URLRegionID) = -2147483648 ? -2147483647 : toInt32(URLRegionID), - toInt16(ResolutionWidth) = -32768 ? -32767 : toInt16(ResolutionWidth), - toInt16(ResolutionHeight) = -32768 ? -32767 : toInt16(ResolutionHeight), - toInt8(ResolutionDepth) = -128 ? -127 : toInt8(ResolutionDepth), - toInt8(FlashMajor) = -128 ? -127 : toInt8(FlashMajor), - toInt8(FlashMinor) = -128 ? -127 : toInt8(FlashMinor), - replaceAll(replaceAll(toValidUTF8(toString(FlashMinor2)), '\n', ' '), '"', '\''), - toInt8(NetMajor) = -128 ? -127 : toInt8(NetMajor), - toInt8(NetMinor) = -128 ? -127 : toInt8(NetMinor), - toInt16(UserAgentMajor) = -32768 ? -32767 : toInt16(UserAgentMajor), - replaceAll(replaceAll(toValidUTF8(toString(UserAgentMinor)), '\n', ' '), '"', '\''), - toInt8(CookieEnable) = -128 ? -127 : toInt8(CookieEnable), - toInt8(JavascriptEnable) = -128 ? -127 : toInt8(JavascriptEnable), - toInt8(IsMobile) = -128 ? -127 : toInt8(IsMobile), - toInt8(MobilePhone) = -128 ? -127 : toInt8(MobilePhone), - replaceAll(replaceAll(toValidUTF8(toString(MobilePhoneModel)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(Params)), '\n', ' '), '"', '\''), - toInt32(IPNetworkID) = -2147483648 ? -2147483647 : toInt32(IPNetworkID), - toInt8(TraficSourceID) = -128 ? -127 : toInt8(TraficSourceID), - toInt16(SearchEngineID) = -32768 ? -32767 : toInt16(SearchEngineID), - replaceAll(replaceAll(toValidUTF8(toString(SearchPhrase)), '\n', ' '), '"', '\''), - toInt8(AdvEngineID) = -128 ? -127 : toInt8(AdvEngineID), - toInt8(IsArtifical) = -128 ? -127 : toInt8(IsArtifical), - toInt16(WindowClientWidth) = -32768 ? -32767 : toInt16(WindowClientWidth), - toInt16(WindowClientHeight) = -32768 ? -32767 : toInt16(WindowClientHeight), - toInt16(ClientTimeZone) = -32768 ? -32767 : toInt16(ClientTimeZone), - ClientEventTime, - toInt8(SilverlightVersion1) = -128 ? -127 : toInt8(SilverlightVersion1), - toInt8(SilverlightVersion2) = -128 ? -127 : toInt8(SilverlightVersion2), - toInt32(SilverlightVersion3) = -2147483648 ? -2147483647 : toInt32(SilverlightVersion3), - toInt16(SilverlightVersion4) = -32768 ? -32767 : toInt16(SilverlightVersion4), - replaceAll(replaceAll(toValidUTF8(toString(PageCharset)), '\n', ' '), '"', '\''), - toInt32(CodeVersion) = -2147483648 ? -2147483647 : toInt32(CodeVersion), - toInt8(IsLink) = -128 ? -127 : toInt8(IsLink), - toInt8(IsDownload) = -128 ? -127 : toInt8(IsDownload), - toInt8(IsNotBounce) = -128 ? -127 : toInt8(IsNotBounce), - toInt64(FUniqID) = -9223372036854775808 ? -9223372036854775807 : toInt64(FUniqID), - replaceAll(replaceAll(toValidUTF8(toString(OriginalURL)), '\n', ' '), '"', '\''), - toInt32(HID) = -2147483648 ? -2147483647 : toInt32(HID), - toInt8(IsOldCounter) = -128 ? -127 : toInt8(IsOldCounter), - toInt8(IsEvent) = -128 ? -127 : toInt8(IsEvent), - toInt8(IsParameter) = -128 ? -127 : toInt8(IsParameter), - toInt8(DontCountHits) = -128 ? -127 : toInt8(DontCountHits), - toInt8(WithHash) = -128 ? -127 : toInt8(WithHash), - replaceAll(replaceAll(toValidUTF8(toString(HitColor)), '\n', ' '), '"', '\''), - LocalEventTime, - toInt8(Age) = -128 ? -127 : toInt8(Age), - toInt8(Sex) = -128 ? -127 : toInt8(Sex), - toInt8(Income) = -128 ? -127 : toInt8(Income), - toInt16(Interests) = -32768 ? -32767 : toInt16(Interests), - toInt8(Robotness) = -128 ? -127 : toInt8(Robotness), - toInt32(RemoteIP) = -2147483648 ? -2147483647 : toInt32(RemoteIP), - toInt32(WindowName) = -2147483648 ? -2147483647 : toInt32(WindowName), - toInt32(OpenerName) = -2147483648 ? -2147483647 : toInt32(OpenerName), - toInt16(HistoryLength) = -32768 ? -32767 : toInt16(HistoryLength), - replaceAll(replaceAll(toValidUTF8(toString(BrowserLanguage)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(BrowserCountry)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(SocialNetwork)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(SocialAction)), '\n', ' '), '"', '\''), - toInt16(HTTPError) = -32768 ? -32767 : toInt16(HTTPError), - toInt32(SendTiming) = -2147483648 ? -2147483647 : toInt32(SendTiming), - toInt32(DNSTiming) = -2147483648 ? -2147483647 : toInt32(DNSTiming), - toInt32(ConnectTiming) = -2147483648 ? -2147483647 : toInt32(ConnectTiming), - toInt32(ResponseStartTiming) = -2147483648 ? -2147483647 : toInt32(ResponseStartTiming), - toInt32(ResponseEndTiming) = -2147483648 ? -2147483647 : toInt32(ResponseEndTiming), - toInt32(FetchTiming) = -2147483648 ? -2147483647 : toInt32(FetchTiming), - toInt8(SocialSourceNetworkID) = -128 ? -127 : toInt8(SocialSourceNetworkID), - replaceAll(replaceAll(toValidUTF8(toString(SocialSourcePage)), '\n', ' '), '"', '\''), - toInt64(ParamPrice) = -9223372036854775808 ? -9223372036854775807 : toInt64(ParamPrice), - replaceAll(replaceAll(toValidUTF8(toString(ParamOrderID)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(ParamCurrency)), '\n', ' '), '"', '\''), - toInt16(ParamCurrencyID) = -32768 ? -32767 : toInt16(ParamCurrencyID), - replaceAll(replaceAll(toValidUTF8(toString(OpenstatServiceName)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(OpenstatCampaignID)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(OpenstatAdID)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(OpenstatSourceID)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(UTMSource)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(UTMMedium)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(UTMCampaign)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(UTMContent)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(UTMTerm)), '\n', ' '), '"', '\''), - replaceAll(replaceAll(toValidUTF8(toString(FromTag)), '\n', ' '), '"', '\''), - toInt8(HasGCLID) = -128 ? -127 : toInt8(HasGCLID), - toInt64(RefererHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(RefererHash), - toInt64(URLHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(URLHash), +SELECT + toInt64(WatchID) = -9223372036854775808 ? -9223372036854775807 : toInt64(WatchID), + toInt8(JavaEnable) = -128 ? -127 : toInt8(JavaEnable), + replaceAll(replaceAll(toValidUTF8(toString(Title)), '\n', ' '), '"', '\''), + toInt16(GoodEvent) = -32768 ? -32767 : toInt16(GoodEvent), + EventTime, + EventDate, + toInt32(CounterID) = -2147483648 ? -2147483647 : toInt32(CounterID), + toInt32(ClientIP) = -2147483648 ? -2147483647 : toInt32(ClientIP), + toInt32(RegionID) = -2147483648 ? -2147483647 : toInt32(RegionID), + toInt64(UserID) = -9223372036854775808 ? -9223372036854775807 : toInt64(UserID), + toInt8(CounterClass) = -128 ? -127 : toInt8(CounterClass), + toInt8(OS) = -128 ? -127 : toInt8(OS), + toInt8(UserAgent) = -128 ? -127 : toInt8(UserAgent), + replaceAll(replaceAll(toValidUTF8(toString(URL)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(Referer)), '\n', ' '), '"', '\''), + toInt8(Refresh) = -128 ? -127 : toInt8(Refresh), + toInt16(RefererCategoryID) = -32768 ? -32767 : toInt16(RefererCategoryID), + toInt32(RefererRegionID) = -2147483648 ? -2147483647 : toInt32(RefererRegionID), + toInt16(URLCategoryID) = -32768 ? -32767 : toInt16(URLCategoryID), + toInt32(URLRegionID) = -2147483648 ? -2147483647 : toInt32(URLRegionID), + toInt16(ResolutionWidth) = -32768 ? -32767 : toInt16(ResolutionWidth), + toInt16(ResolutionHeight) = -32768 ? -32767 : toInt16(ResolutionHeight), + toInt8(ResolutionDepth) = -128 ? -127 : toInt8(ResolutionDepth), + toInt8(FlashMajor) = -128 ? -127 : toInt8(FlashMajor), + toInt8(FlashMinor) = -128 ? -127 : toInt8(FlashMinor), + replaceAll(replaceAll(toValidUTF8(toString(FlashMinor2)), '\n', ' '), '"', '\''), + toInt8(NetMajor) = -128 ? -127 : toInt8(NetMajor), + toInt8(NetMinor) = -128 ? -127 : toInt8(NetMinor), + toInt16(UserAgentMajor) = -32768 ? -32767 : toInt16(UserAgentMajor), + replaceAll(replaceAll(toValidUTF8(toString(UserAgentMinor)), '\n', ' '), '"', '\''), + toInt8(CookieEnable) = -128 ? -127 : toInt8(CookieEnable), + toInt8(JavascriptEnable) = -128 ? -127 : toInt8(JavascriptEnable), + toInt8(IsMobile) = -128 ? -127 : toInt8(IsMobile), + toInt8(MobilePhone) = -128 ? -127 : toInt8(MobilePhone), + replaceAll(replaceAll(toValidUTF8(toString(MobilePhoneModel)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(Params)), '\n', ' '), '"', '\''), + toInt32(IPNetworkID) = -2147483648 ? -2147483647 : toInt32(IPNetworkID), + toInt8(TraficSourceID) = -128 ? -127 : toInt8(TraficSourceID), + toInt16(SearchEngineID) = -32768 ? -32767 : toInt16(SearchEngineID), + replaceAll(replaceAll(toValidUTF8(toString(SearchPhrase)), '\n', ' '), '"', '\''), + toInt8(AdvEngineID) = -128 ? -127 : toInt8(AdvEngineID), + toInt8(IsArtifical) = -128 ? -127 : toInt8(IsArtifical), + toInt16(WindowClientWidth) = -32768 ? -32767 : toInt16(WindowClientWidth), + toInt16(WindowClientHeight) = -32768 ? -32767 : toInt16(WindowClientHeight), + toInt16(ClientTimeZone) = -32768 ? -32767 : toInt16(ClientTimeZone), + ClientEventTime, + toInt8(SilverlightVersion1) = -128 ? -127 : toInt8(SilverlightVersion1), + toInt8(SilverlightVersion2) = -128 ? -127 : toInt8(SilverlightVersion2), + toInt32(SilverlightVersion3) = -2147483648 ? -2147483647 : toInt32(SilverlightVersion3), + toInt16(SilverlightVersion4) = -32768 ? -32767 : toInt16(SilverlightVersion4), + replaceAll(replaceAll(toValidUTF8(toString(PageCharset)), '\n', ' '), '"', '\''), + toInt32(CodeVersion) = -2147483648 ? -2147483647 : toInt32(CodeVersion), + toInt8(IsLink) = -128 ? -127 : toInt8(IsLink), + toInt8(IsDownload) = -128 ? -127 : toInt8(IsDownload), + toInt8(IsNotBounce) = -128 ? -127 : toInt8(IsNotBounce), + toInt64(FUniqID) = -9223372036854775808 ? -9223372036854775807 : toInt64(FUniqID), + replaceAll(replaceAll(toValidUTF8(toString(OriginalURL)), '\n', ' '), '"', '\''), + toInt32(HID) = -2147483648 ? -2147483647 : toInt32(HID), + toInt8(IsOldCounter) = -128 ? -127 : toInt8(IsOldCounter), + toInt8(IsEvent) = -128 ? -127 : toInt8(IsEvent), + toInt8(IsParameter) = -128 ? -127 : toInt8(IsParameter), + toInt8(DontCountHits) = -128 ? -127 : toInt8(DontCountHits), + toInt8(WithHash) = -128 ? -127 : toInt8(WithHash), + replaceAll(replaceAll(toValidUTF8(toString(HitColor)), '\n', ' '), '"', '\''), + LocalEventTime, + toInt8(Age) = -128 ? -127 : toInt8(Age), + toInt8(Sex) = -128 ? -127 : toInt8(Sex), + toInt8(Income) = -128 ? -127 : toInt8(Income), + toInt16(Interests) = -32768 ? -32767 : toInt16(Interests), + toInt8(Robotness) = -128 ? -127 : toInt8(Robotness), + toInt32(RemoteIP) = -2147483648 ? -2147483647 : toInt32(RemoteIP), + toInt32(WindowName) = -2147483648 ? -2147483647 : toInt32(WindowName), + toInt32(OpenerName) = -2147483648 ? -2147483647 : toInt32(OpenerName), + toInt16(HistoryLength) = -32768 ? -32767 : toInt16(HistoryLength), + replaceAll(replaceAll(toValidUTF8(toString(BrowserLanguage)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(BrowserCountry)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(SocialNetwork)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(SocialAction)), '\n', ' '), '"', '\''), + toInt16(HTTPError) = -32768 ? -32767 : toInt16(HTTPError), + toInt32(SendTiming) = -2147483648 ? -2147483647 : toInt32(SendTiming), + toInt32(DNSTiming) = -2147483648 ? -2147483647 : toInt32(DNSTiming), + toInt32(ConnectTiming) = -2147483648 ? -2147483647 : toInt32(ConnectTiming), + toInt32(ResponseStartTiming) = -2147483648 ? -2147483647 : toInt32(ResponseStartTiming), + toInt32(ResponseEndTiming) = -2147483648 ? -2147483647 : toInt32(ResponseEndTiming), + toInt32(FetchTiming) = -2147483648 ? -2147483647 : toInt32(FetchTiming), + toInt8(SocialSourceNetworkID) = -128 ? -127 : toInt8(SocialSourceNetworkID), + replaceAll(replaceAll(toValidUTF8(toString(SocialSourcePage)), '\n', ' '), '"', '\''), + toInt64(ParamPrice) = -9223372036854775808 ? -9223372036854775807 : toInt64(ParamPrice), + replaceAll(replaceAll(toValidUTF8(toString(ParamOrderID)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(ParamCurrency)), '\n', ' '), '"', '\''), + toInt16(ParamCurrencyID) = -32768 ? -32767 : toInt16(ParamCurrencyID), + replaceAll(replaceAll(toValidUTF8(toString(OpenstatServiceName)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(OpenstatCampaignID)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(OpenstatAdID)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(OpenstatSourceID)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(UTMSource)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(UTMMedium)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(UTMCampaign)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(UTMContent)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(UTMTerm)), '\n', ' '), '"', '\''), + replaceAll(replaceAll(toValidUTF8(toString(FromTag)), '\n', ' '), '"', '\''), + toInt8(HasGCLID) = -128 ? -127 : toInt8(HasGCLID), + toInt64(RefererHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(RefererHash), + toInt64(URLHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(URLHash), toInt32(CLID) = -2147483648 ? -2147483647 : toInt32(CLID) FROM hits_100m_obfuscated INTO OUTFILE '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' @@ -668,7 +668,7 @@ COPY 100000000 RECORDS INTO hits FROM '/home/milovidov/example_datasets/hits_100 Does not work. ``` -Failed to import table 'hits', +Failed to import table 'hits', clk: 1.091 sec ``` @@ -920,118 +920,118 @@ COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monet Nothing good happened, it failed after 2.5 minutes with incomprehensible error: ``` -Failed to import table 'hits', +Failed to import table 'hits', clk: 2:30 min ``` Let's replace all backslashes from CSV. ``` -SELECT - toInt64(WatchID) = -9223372036854775808 ? -9223372036854775807 : toInt64(WatchID), - toInt8(JavaEnable) = -128 ? -127 : toInt8(JavaEnable), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(Title)), '\n', ' '), '"', '\''), '\\', '/'), - toInt16(GoodEvent) = -32768 ? -32767 : toInt16(GoodEvent), - EventTime, - EventDate, - toInt32(CounterID) = -2147483648 ? -2147483647 : toInt32(CounterID), - toInt32(ClientIP) = -2147483648 ? -2147483647 : toInt32(ClientIP), - toInt32(RegionID) = -2147483648 ? -2147483647 : toInt32(RegionID), - toInt64(UserID) = -9223372036854775808 ? -9223372036854775807 : toInt64(UserID), - toInt8(CounterClass) = -128 ? -127 : toInt8(CounterClass), - toInt8(OS) = -128 ? -127 : toInt8(OS), - toInt8(UserAgent) = -128 ? -127 : toInt8(UserAgent), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(URL)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(Referer)), '\n', ' '), '"', '\''), '\\', '/'), - toInt8(Refresh) = -128 ? -127 : toInt8(Refresh), - toInt16(RefererCategoryID) = -32768 ? -32767 : toInt16(RefererCategoryID), - toInt32(RefererRegionID) = -2147483648 ? -2147483647 : toInt32(RefererRegionID), - toInt16(URLCategoryID) = -32768 ? -32767 : toInt16(URLCategoryID), - toInt32(URLRegionID) = -2147483648 ? -2147483647 : toInt32(URLRegionID), - toInt16(ResolutionWidth) = -32768 ? -32767 : toInt16(ResolutionWidth), - toInt16(ResolutionHeight) = -32768 ? -32767 : toInt16(ResolutionHeight), - toInt8(ResolutionDepth) = -128 ? -127 : toInt8(ResolutionDepth), - toInt8(FlashMajor) = -128 ? -127 : toInt8(FlashMajor), - toInt8(FlashMinor) = -128 ? -127 : toInt8(FlashMinor), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(FlashMinor2)), '\n', ' '), '"', '\''), '\\', '/'), - toInt8(NetMajor) = -128 ? -127 : toInt8(NetMajor), - toInt8(NetMinor) = -128 ? -127 : toInt8(NetMinor), - toInt16(UserAgentMajor) = -32768 ? -32767 : toInt16(UserAgentMajor), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UserAgentMinor)), '\n', ' '), '"', '\''), '\\', '/'), - toInt8(CookieEnable) = -128 ? -127 : toInt8(CookieEnable), - toInt8(JavascriptEnable) = -128 ? -127 : toInt8(JavascriptEnable), - toInt8(IsMobile) = -128 ? -127 : toInt8(IsMobile), - toInt8(MobilePhone) = -128 ? -127 : toInt8(MobilePhone), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(MobilePhoneModel)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(Params)), '\n', ' '), '"', '\''), '\\', '/'), - toInt32(IPNetworkID) = -2147483648 ? -2147483647 : toInt32(IPNetworkID), - toInt8(TraficSourceID) = -128 ? -127 : toInt8(TraficSourceID), - toInt16(SearchEngineID) = -32768 ? -32767 : toInt16(SearchEngineID), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(SearchPhrase)), '\n', ' '), '"', '\''), '\\', '/'), - toInt8(AdvEngineID) = -128 ? -127 : toInt8(AdvEngineID), - toInt8(IsArtifical) = -128 ? -127 : toInt8(IsArtifical), - toInt16(WindowClientWidth) = -32768 ? -32767 : toInt16(WindowClientWidth), - toInt16(WindowClientHeight) = -32768 ? -32767 : toInt16(WindowClientHeight), - toInt16(ClientTimeZone) = -32768 ? -32767 : toInt16(ClientTimeZone), - ClientEventTime, - toInt8(SilverlightVersion1) = -128 ? -127 : toInt8(SilverlightVersion1), - toInt8(SilverlightVersion2) = -128 ? -127 : toInt8(SilverlightVersion2), - toInt32(SilverlightVersion3) = -2147483648 ? -2147483647 : toInt32(SilverlightVersion3), - toInt16(SilverlightVersion4) = -32768 ? -32767 : toInt16(SilverlightVersion4), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(PageCharset)), '\n', ' '), '"', '\''), '\\', '/'), - toInt32(CodeVersion) = -2147483648 ? -2147483647 : toInt32(CodeVersion), - toInt8(IsLink) = -128 ? -127 : toInt8(IsLink), - toInt8(IsDownload) = -128 ? -127 : toInt8(IsDownload), - toInt8(IsNotBounce) = -128 ? -127 : toInt8(IsNotBounce), - toInt64(FUniqID) = -9223372036854775808 ? -9223372036854775807 : toInt64(FUniqID), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OriginalURL)), '\n', ' '), '"', '\''), '\\', '/'), - toInt32(HID) = -2147483648 ? -2147483647 : toInt32(HID), - toInt8(IsOldCounter) = -128 ? -127 : toInt8(IsOldCounter), - toInt8(IsEvent) = -128 ? -127 : toInt8(IsEvent), - toInt8(IsParameter) = -128 ? -127 : toInt8(IsParameter), - toInt8(DontCountHits) = -128 ? -127 : toInt8(DontCountHits), - toInt8(WithHash) = -128 ? -127 : toInt8(WithHash), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(HitColor)), '\n', ' '), '"', '\''), '\\', '/'), - LocalEventTime, - toInt8(Age) = -128 ? -127 : toInt8(Age), - toInt8(Sex) = -128 ? -127 : toInt8(Sex), - toInt8(Income) = -128 ? -127 : toInt8(Income), - toInt16(Interests) = -32768 ? -32767 : toInt16(Interests), - toInt8(Robotness) = -128 ? -127 : toInt8(Robotness), - toInt32(RemoteIP) = -2147483648 ? -2147483647 : toInt32(RemoteIP), - toInt32(WindowName) = -2147483648 ? -2147483647 : toInt32(WindowName), - toInt32(OpenerName) = -2147483648 ? -2147483647 : toInt32(OpenerName), - toInt16(HistoryLength) = -32768 ? -32767 : toInt16(HistoryLength), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(BrowserLanguage)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(BrowserCountry)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(SocialNetwork)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(SocialAction)), '\n', ' '), '"', '\''), '\\', '/'), - toInt16(HTTPError) = -32768 ? -32767 : toInt16(HTTPError), - toInt32(SendTiming) = -2147483648 ? -2147483647 : toInt32(SendTiming), - toInt32(DNSTiming) = -2147483648 ? -2147483647 : toInt32(DNSTiming), - toInt32(ConnectTiming) = -2147483648 ? -2147483647 : toInt32(ConnectTiming), - toInt32(ResponseStartTiming) = -2147483648 ? -2147483647 : toInt32(ResponseStartTiming), - toInt32(ResponseEndTiming) = -2147483648 ? -2147483647 : toInt32(ResponseEndTiming), - toInt32(FetchTiming) = -2147483648 ? -2147483647 : toInt32(FetchTiming), - toInt8(SocialSourceNetworkID) = -128 ? -127 : toInt8(SocialSourceNetworkID), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(SocialSourcePage)), '\n', ' '), '"', '\''), '\\', '/'), - toInt64(ParamPrice) = -9223372036854775808 ? -9223372036854775807 : toInt64(ParamPrice), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(ParamOrderID)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(ParamCurrency)), '\n', ' '), '"', '\''), '\\', '/'), - toInt16(ParamCurrencyID) = -32768 ? -32767 : toInt16(ParamCurrencyID), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OpenstatServiceName)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OpenstatCampaignID)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OpenstatAdID)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OpenstatSourceID)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMSource)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMMedium)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMCampaign)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMContent)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMTerm)), '\n', ' '), '"', '\''), '\\', '/'), - replaceAll(replaceAll(replaceAll(toValidUTF8(toString(FromTag)), '\n', ' '), '"', '\''), '\\', '/'), - toInt8(HasGCLID) = -128 ? -127 : toInt8(HasGCLID), - toInt64(RefererHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(RefererHash), - toInt64(URLHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(URLHash), +SELECT + toInt64(WatchID) = -9223372036854775808 ? -9223372036854775807 : toInt64(WatchID), + toInt8(JavaEnable) = -128 ? -127 : toInt8(JavaEnable), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(Title)), '\n', ' '), '"', '\''), '\\', '/'), + toInt16(GoodEvent) = -32768 ? -32767 : toInt16(GoodEvent), + EventTime, + EventDate, + toInt32(CounterID) = -2147483648 ? -2147483647 : toInt32(CounterID), + toInt32(ClientIP) = -2147483648 ? -2147483647 : toInt32(ClientIP), + toInt32(RegionID) = -2147483648 ? -2147483647 : toInt32(RegionID), + toInt64(UserID) = -9223372036854775808 ? -9223372036854775807 : toInt64(UserID), + toInt8(CounterClass) = -128 ? -127 : toInt8(CounterClass), + toInt8(OS) = -128 ? -127 : toInt8(OS), + toInt8(UserAgent) = -128 ? -127 : toInt8(UserAgent), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(URL)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(Referer)), '\n', ' '), '"', '\''), '\\', '/'), + toInt8(Refresh) = -128 ? -127 : toInt8(Refresh), + toInt16(RefererCategoryID) = -32768 ? -32767 : toInt16(RefererCategoryID), + toInt32(RefererRegionID) = -2147483648 ? -2147483647 : toInt32(RefererRegionID), + toInt16(URLCategoryID) = -32768 ? -32767 : toInt16(URLCategoryID), + toInt32(URLRegionID) = -2147483648 ? -2147483647 : toInt32(URLRegionID), + toInt16(ResolutionWidth) = -32768 ? -32767 : toInt16(ResolutionWidth), + toInt16(ResolutionHeight) = -32768 ? -32767 : toInt16(ResolutionHeight), + toInt8(ResolutionDepth) = -128 ? -127 : toInt8(ResolutionDepth), + toInt8(FlashMajor) = -128 ? -127 : toInt8(FlashMajor), + toInt8(FlashMinor) = -128 ? -127 : toInt8(FlashMinor), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(FlashMinor2)), '\n', ' '), '"', '\''), '\\', '/'), + toInt8(NetMajor) = -128 ? -127 : toInt8(NetMajor), + toInt8(NetMinor) = -128 ? -127 : toInt8(NetMinor), + toInt16(UserAgentMajor) = -32768 ? -32767 : toInt16(UserAgentMajor), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UserAgentMinor)), '\n', ' '), '"', '\''), '\\', '/'), + toInt8(CookieEnable) = -128 ? -127 : toInt8(CookieEnable), + toInt8(JavascriptEnable) = -128 ? -127 : toInt8(JavascriptEnable), + toInt8(IsMobile) = -128 ? -127 : toInt8(IsMobile), + toInt8(MobilePhone) = -128 ? -127 : toInt8(MobilePhone), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(MobilePhoneModel)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(Params)), '\n', ' '), '"', '\''), '\\', '/'), + toInt32(IPNetworkID) = -2147483648 ? -2147483647 : toInt32(IPNetworkID), + toInt8(TraficSourceID) = -128 ? -127 : toInt8(TraficSourceID), + toInt16(SearchEngineID) = -32768 ? -32767 : toInt16(SearchEngineID), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(SearchPhrase)), '\n', ' '), '"', '\''), '\\', '/'), + toInt8(AdvEngineID) = -128 ? -127 : toInt8(AdvEngineID), + toInt8(IsArtifical) = -128 ? -127 : toInt8(IsArtifical), + toInt16(WindowClientWidth) = -32768 ? -32767 : toInt16(WindowClientWidth), + toInt16(WindowClientHeight) = -32768 ? -32767 : toInt16(WindowClientHeight), + toInt16(ClientTimeZone) = -32768 ? -32767 : toInt16(ClientTimeZone), + ClientEventTime, + toInt8(SilverlightVersion1) = -128 ? -127 : toInt8(SilverlightVersion1), + toInt8(SilverlightVersion2) = -128 ? -127 : toInt8(SilverlightVersion2), + toInt32(SilverlightVersion3) = -2147483648 ? -2147483647 : toInt32(SilverlightVersion3), + toInt16(SilverlightVersion4) = -32768 ? -32767 : toInt16(SilverlightVersion4), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(PageCharset)), '\n', ' '), '"', '\''), '\\', '/'), + toInt32(CodeVersion) = -2147483648 ? -2147483647 : toInt32(CodeVersion), + toInt8(IsLink) = -128 ? -127 : toInt8(IsLink), + toInt8(IsDownload) = -128 ? -127 : toInt8(IsDownload), + toInt8(IsNotBounce) = -128 ? -127 : toInt8(IsNotBounce), + toInt64(FUniqID) = -9223372036854775808 ? -9223372036854775807 : toInt64(FUniqID), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OriginalURL)), '\n', ' '), '"', '\''), '\\', '/'), + toInt32(HID) = -2147483648 ? -2147483647 : toInt32(HID), + toInt8(IsOldCounter) = -128 ? -127 : toInt8(IsOldCounter), + toInt8(IsEvent) = -128 ? -127 : toInt8(IsEvent), + toInt8(IsParameter) = -128 ? -127 : toInt8(IsParameter), + toInt8(DontCountHits) = -128 ? -127 : toInt8(DontCountHits), + toInt8(WithHash) = -128 ? -127 : toInt8(WithHash), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(HitColor)), '\n', ' '), '"', '\''), '\\', '/'), + LocalEventTime, + toInt8(Age) = -128 ? -127 : toInt8(Age), + toInt8(Sex) = -128 ? -127 : toInt8(Sex), + toInt8(Income) = -128 ? -127 : toInt8(Income), + toInt16(Interests) = -32768 ? -32767 : toInt16(Interests), + toInt8(Robotness) = -128 ? -127 : toInt8(Robotness), + toInt32(RemoteIP) = -2147483648 ? -2147483647 : toInt32(RemoteIP), + toInt32(WindowName) = -2147483648 ? -2147483647 : toInt32(WindowName), + toInt32(OpenerName) = -2147483648 ? -2147483647 : toInt32(OpenerName), + toInt16(HistoryLength) = -32768 ? -32767 : toInt16(HistoryLength), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(BrowserLanguage)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(BrowserCountry)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(SocialNetwork)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(SocialAction)), '\n', ' '), '"', '\''), '\\', '/'), + toInt16(HTTPError) = -32768 ? -32767 : toInt16(HTTPError), + toInt32(SendTiming) = -2147483648 ? -2147483647 : toInt32(SendTiming), + toInt32(DNSTiming) = -2147483648 ? -2147483647 : toInt32(DNSTiming), + toInt32(ConnectTiming) = -2147483648 ? -2147483647 : toInt32(ConnectTiming), + toInt32(ResponseStartTiming) = -2147483648 ? -2147483647 : toInt32(ResponseStartTiming), + toInt32(ResponseEndTiming) = -2147483648 ? -2147483647 : toInt32(ResponseEndTiming), + toInt32(FetchTiming) = -2147483648 ? -2147483647 : toInt32(FetchTiming), + toInt8(SocialSourceNetworkID) = -128 ? -127 : toInt8(SocialSourceNetworkID), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(SocialSourcePage)), '\n', ' '), '"', '\''), '\\', '/'), + toInt64(ParamPrice) = -9223372036854775808 ? -9223372036854775807 : toInt64(ParamPrice), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(ParamOrderID)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(ParamCurrency)), '\n', ' '), '"', '\''), '\\', '/'), + toInt16(ParamCurrencyID) = -32768 ? -32767 : toInt16(ParamCurrencyID), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OpenstatServiceName)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OpenstatCampaignID)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OpenstatAdID)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(OpenstatSourceID)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMSource)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMMedium)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMCampaign)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMContent)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(UTMTerm)), '\n', ' '), '"', '\''), '\\', '/'), + replaceAll(replaceAll(replaceAll(toValidUTF8(toString(FromTag)), '\n', ' '), '"', '\''), '\\', '/'), + toInt8(HasGCLID) = -128 ? -127 : toInt8(HasGCLID), + toInt64(RefererHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(RefererHash), + toInt64(URLHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(URLHash), toInt32(CLID) = -2147483648 ? -2147483647 : toInt32(CLID) FROM hits_100m_obfuscated INTO OUTFILE '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' @@ -1048,14 +1048,14 @@ And there are strange pauses... ``` sql>COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.csv' USING DELIMITERS ',', '\n', '"'; -Failed to import table 'hits', +Failed to import table 'hits', clk: 2:14 min ``` It still does not work!!! Let's look at the logs. -Logs are found in +Logs are found in ``` /var/log/monetdb$ sudo less merovingian.log @@ -1212,3 +1212,11 @@ awk '{ else if (i % 3 == 2) { c = $1; print "[" a ", " b ", " c "]," }; ++i; }' < tmp.txt ``` + +When I run: + +``` +sudo systemctl stop monetdbd +``` + +It takes a few minutes to complete. From 5ecb9c1988d81ee0d7a023eaa3093ec208d7be70 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed, 12 Aug 2020 21:31:26 +0300 Subject: [PATCH 295/374] Fix a bug on the website: bad place of horizontal scroll bar --- website/benchmark/dbms/index.html | 2 +- website/benchmark/hardware/index.html | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/website/benchmark/dbms/index.html b/website/benchmark/dbms/index.html index 43eb7ac7531..28baa33dcdc 100644 --- a/website/benchmark/dbms/index.html +++ b/website/benchmark/dbms/index.html @@ -35,7 +35,7 @@ <div class="row mb-3"> <div class="col"> <h3 class="my-3">Full results</h3> - <div id="comparison_table" class="table-responsive"></div> + <div id="comparison_table"></div> </div> </div> diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index 6c6b4573f77..ce7fb602258 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -35,7 +35,7 @@ <div class="row mb-3"> <div class="col"> <h3 class="my-3">Full results</h3> - <div id="comparison_table" class="table-responsive"></div> + <div id="comparison_table"></div> </div> </div> From 96d10a76d8442270b72fcacbbfe4e66f6ed1289f Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov <vzakaznikov@protonmail.com> Date: Wed, 12 Aug 2020 16:30:26 -0400 Subject: [PATCH 296/374] Updating LDAP user authentication suite to check user authentications when users are added using RBAC. --- tests/testflows/ldap/tests/authentications.py | 96 ++++++++++--------- tests/testflows/ldap/tests/common.py | 34 +++++-- 2 files changed, 79 insertions(+), 51 deletions(-) diff --git a/tests/testflows/ldap/tests/authentications.py b/tests/testflows/ldap/tests/authentications.py index 4b054fb694e..949f8444c21 100644 --- a/tests/testflows/ldap/tests/authentications.py +++ b/tests/testflows/ldap/tests/authentications.py @@ -34,7 +34,7 @@ def login_and_execute_query(self, username, password, exitcode=None, message=Non message=message, steps=steps) @TestScenario -def add_user_to_ldap_and_login(self, server, user=None, ch_user=None, login=None, exitcode=None, message=None): +def add_user_to_ldap_and_login(self, server, user=None, ch_user=None, login=None, exitcode=None, message=None, rbac=False): """Add user to LDAP and ClickHouse and then try to login.""" self.context.ldap_node = self.context.cluster.node(server) @@ -49,7 +49,7 @@ def add_user_to_ldap_and_login(self, server, user=None, ch_user=None, login=None ch_user["username"] = ch_user.get("username", user["cn"]) ch_user["server"] = ch_user.get("server", user["_server"]) - with ldap_authenticated_users(ch_user, config_file=f"ldap_users_{getuid()}.xml", restart=True): + with ldap_authenticated_users(ch_user, config_file=f"ldap_users_{getuid()}.xml", restart=True, rbac=rbac): username = login.get("username", user["cn"]) password = login.get("password", user["userpassword"]) login_and_execute_query(username=username, password=password, exitcode=exitcode, message=message) @@ -59,7 +59,7 @@ def add_user_to_ldap_and_login(self, server, user=None, ch_user=None, login=None RQ_SRS_007_LDAP_Authentication_Parallel("1.0"), RQ_SRS_007_LDAP_Authentication_Parallel_ValidAndInvalid("1.0") ) -def parallel_login(self, server, user_count=10, timeout=200): +def parallel_login(self, server, user_count=10, timeout=200, rbac=False): """Check that login of valid and invalid LDAP authenticated users works in parallel.""" self.context.ldap_node = self.context.cluster.node(server) user = None @@ -67,7 +67,7 @@ def parallel_login(self, server, user_count=10, timeout=200): users = [{"cn": f"parallel_user{i}", "userpassword": randomword(20)} for i in range(user_count)] with ldap_users(*users): - with ldap_authenticated_users(*[{"username": user["cn"], "server": server} for user in users]): + with ldap_authenticated_users(*[{"username": user["cn"], "server": server} for user in users], rbac=rbac): def login_with_valid_username_and_password(users, i, iterations=10): with When(f"valid users try to login #{i}"): @@ -113,7 +113,7 @@ def parallel_login(self, server, user_count=10, timeout=200): RQ_SRS_007_LDAP_Authentication_Invalid("1.0"), RQ_SRS_007_LDAP_Authentication_Invalid_DeletedUser("1.0") ) -def login_after_user_is_deleted_from_ldap(self, server): +def login_after_user_is_deleted_from_ldap(self, server, rbac=False): """Check that login fails after user is deleted from LDAP.""" self.context.ldap_node = self.context.cluster.node(server) user = None @@ -123,7 +123,8 @@ def login_after_user_is_deleted_from_ldap(self, server): user = {"cn": "myuser", "userpassword": "myuser"} user = add_user_to_ldap(**user) - with ldap_authenticated_users({"username": user["cn"], "server": server}, config_file=f"ldap_users_{getuid()}.xml", restart=True): + with ldap_authenticated_users({"username": user["cn"], "server": server}, config_file=f"ldap_users_{getuid()}.xml", + restart=True, rbac=rbac): login_and_execute_query(username=user["cn"], password=user["userpassword"]) with When("I delete this user from LDAP"): @@ -144,7 +145,7 @@ def login_after_user_is_deleted_from_ldap(self, server): RQ_SRS_007_LDAP_Authentication_Invalid("1.0"), RQ_SRS_007_LDAP_Authentication_PasswordChanged("1.0") ) -def login_after_user_password_changed_in_ldap(self, server): +def login_after_user_password_changed_in_ldap(self, server, rbac=False): """Check that login fails after user password is changed in LDAP.""" self.context.ldap_node = self.context.cluster.node(server) user = None @@ -154,7 +155,8 @@ def login_after_user_password_changed_in_ldap(self, server): user = {"cn": "myuser", "userpassword": "myuser"} user = add_user_to_ldap(**user) - with ldap_authenticated_users({"username": user["cn"], "server": server}, config_file=f"ldap_users_{getuid()}.xml", restart=True): + with ldap_authenticated_users({"username": user["cn"], "server": server}, config_file=f"ldap_users_{getuid()}.xml", + restart=True, rbac=rbac): login_and_execute_query(username=user["cn"], password=user["userpassword"]) with When("I change user password in LDAP"): @@ -179,7 +181,7 @@ def login_after_user_password_changed_in_ldap(self, server): RQ_SRS_007_LDAP_Authentication_Invalid("1.0"), RQ_SRS_007_LDAP_Authentication_UsernameChanged("1.0") ) -def login_after_user_cn_changed_in_ldap(self, server): +def login_after_user_cn_changed_in_ldap(self, server, rbac=False): """Check that login fails after user cn is changed in LDAP.""" self.context.ldap_node = self.context.cluster.node(server) user = None @@ -189,7 +191,8 @@ def login_after_user_cn_changed_in_ldap(self, server): user = {"cn": "myuser", "userpassword": "myuser"} user = add_user_to_ldap(**user) - with ldap_authenticated_users({"username": user["cn"], "server": server}, config_file=f"ldap_users_{getuid()}.xml", restart=True): + with ldap_authenticated_users({"username": user["cn"], "server": server}, + config_file=f"ldap_users_{getuid()}.xml", restart=True, rbac=rbac): login_and_execute_query(username=user["cn"], password=user["userpassword"]) with When("I change user password in LDAP"): @@ -210,7 +213,7 @@ def login_after_user_cn_changed_in_ldap(self, server): RQ_SRS_007_LDAP_Authentication_Valid("1.0"), RQ_SRS_007_LDAP_Authentication_LDAPServerRestart("1.0") ) -def login_after_ldap_server_is_restarted(self, server, timeout=60): +def login_after_ldap_server_is_restarted(self, server, timeout=60, rbac=False): """Check that login succeeds after LDAP server is restarted.""" self.context.ldap_node = self.context.cluster.node(server) user = None @@ -220,7 +223,7 @@ def login_after_ldap_server_is_restarted(self, server, timeout=60): user = {"cn": "myuser", "userpassword": getuid()} user = add_user_to_ldap(**user) - with ldap_authenticated_users({"username": user["cn"], "server": server}): + with ldap_authenticated_users({"username": user["cn"], "server": server}, rbac=rbac): login_and_execute_query(username=user["cn"], password=user["userpassword"]) with When("I restart LDAP server"): @@ -245,7 +248,7 @@ def login_after_ldap_server_is_restarted(self, server, timeout=60): RQ_SRS_007_LDAP_Authentication_Valid("1.0"), RQ_SRS_007_LDAP_Authentication_ClickHouseServerRestart("1.0") ) -def login_after_clickhouse_server_is_restarted(self, server, timeout=60): +def login_after_clickhouse_server_is_restarted(self, server, timeout=60, rbac=False): """Check that login succeeds after ClickHouse server is restarted.""" self.context.ldap_node = self.context.cluster.node(server) user = None @@ -255,7 +258,7 @@ def login_after_clickhouse_server_is_restarted(self, server, timeout=60): user = {"cn": "myuser", "userpassword": getuid()} user = add_user_to_ldap(**user) - with ldap_authenticated_users({"username": user["cn"], "server": server}): + with ldap_authenticated_users({"username": user["cn"], "server": server}, rbac=rbac): login_and_execute_query(username=user["cn"], password=user["userpassword"]) with When("I restart ClickHouse server"): @@ -280,20 +283,20 @@ def login_after_clickhouse_server_is_restarted(self, server, timeout=60): RQ_SRS_007_LDAP_Authentication_Invalid("1.0"), RQ_SRS_007_LDAP_Authentication_Password_Empty("1.0") ) -def valid_username_with_valid_empty_password(self, server): +def valid_username_with_valid_empty_password(self, server, rbac=False): """Check that we can't login using valid username that has empty password.""" user = {"cn": "empty_password", "userpassword": ""} exitcode = 4 message = f"DB::Exception: {user['cn']}: Authentication failed: password is incorrect or there is no user with such name" - add_user_to_ldap_and_login(user=user, exitcode=exitcode, message=message, server=server) + add_user_to_ldap_and_login(user=user, exitcode=exitcode, message=message, server=server, rbac=rbac) @TestScenario @Requirements( RQ_SRS_007_LDAP_Authentication_Invalid("1.0"), RQ_SRS_007_LDAP_Authentication_Password_Empty("1.0") ) -def valid_username_and_invalid_empty_password(self, server): +def valid_username_and_invalid_empty_password(self, server, rbac=False): """Check that we can't login using valid username but invalid empty password.""" username = "user_non_empty_password" user = {"cn": username, "userpassword": username} @@ -302,25 +305,25 @@ def valid_username_and_invalid_empty_password(self, server): exitcode = 4 message = f"DB::Exception: {username}: Authentication failed: password is incorrect or there is no user with such name" - add_user_to_ldap_and_login(user=user, login=login, exitcode=exitcode, message=message, server=server) + add_user_to_ldap_and_login(user=user, login=login, exitcode=exitcode, message=message, server=server, rbac=rbac) @TestScenario @Requirements( RQ_SRS_007_LDAP_Authentication_Valid("1.0") ) -def valid_username_and_password(self, server): +def valid_username_and_password(self, server, rbac=False): """Check that we can login using valid username and password.""" username = "valid_username_and_password" user = {"cn": username, "userpassword": username} with When(f"I add user {username} to LDAP and try to login"): - add_user_to_ldap_and_login(user=user, server=server) + add_user_to_ldap_and_login(user=user, server=server, rbac=rbac) @TestScenario @Requirements( RQ_SRS_007_LDAP_Authentication_Invalid("1.0") ) -def valid_username_and_password_invalid_server(self, server=None): +def valid_username_and_password_invalid_server(self, server=None, rbac=False): """Check that we can't login using valid username and valid password but for a different server.""" self.context.ldap_node = self.context.cluster.node("openldap1") @@ -330,8 +333,8 @@ def valid_username_and_password_invalid_server(self, server=None): exitcode = 4 message = f"DB::Exception: user2: Authentication failed: password is incorrect or there is no user with such name" - with ldap_authenticated_users(user, config_file=f"ldap_users_{getuid()}.xml", restart=True): - login_and_execute_query(username="user2", password="user2", exitcode=exitcode, message=message) + with ldap_authenticated_users(user, config_file=f"ldap_users_{getuid()}.xml", restart=True, rbac=rbac): + login_and_execute_query(username="user2", password="user2", exitcode=exitcode, message=message) @TestScenario @Requirements( @@ -339,18 +342,18 @@ def valid_username_and_password_invalid_server(self, server=None): RQ_SRS_007_LDAP_Authentication_Username_Long("1.0"), RQ_SRS_007_LDAP_Configuration_User_Name_Long("1.0") ) -def valid_long_username_and_short_password(self, server): +def valid_long_username_and_short_password(self, server, rbac=False): """Check that we can login using valid very long username and short password.""" username = "long_username_12345678901234567890123456789012345678901234567890123456789012345678901234567890" user = {"cn": username, "userpassword": "long_username"} - add_user_to_ldap_and_login(user=user, server=server) + add_user_to_ldap_and_login(user=user, server=server, rbac=rbac) @TestScenario @Requirements( RQ_SRS_007_LDAP_Authentication_Invalid("1.0") ) -def invalid_long_username_and_valid_short_password(self, server): +def invalid_long_username_and_valid_short_password(self, server, rbac=False): """Check that we can't login using slightly invalid long username but valid password.""" username = "long_username_12345678901234567890123456789012345678901234567890123456789012345678901234567890" user = {"cn": username, "userpassword": "long_username"} @@ -359,24 +362,24 @@ def invalid_long_username_and_valid_short_password(self, server): exitcode = 4 message=f"DB::Exception: {login['username']}: Authentication failed: password is incorrect or there is no user with such name" - add_user_to_ldap_and_login(user=user, login=login, exitcode=exitcode, message=message, server=server) + add_user_to_ldap_and_login(user=user, login=login, exitcode=exitcode, message=message, server=server, rbac=rbac) @TestScenario @Requirements( RQ_SRS_007_LDAP_Authentication_Valid("1.0"), RQ_SRS_007_LDAP_Authentication_Password_Long("1.0") ) -def valid_short_username_and_long_password(self, server): +def valid_short_username_and_long_password(self, server, rbac=False): """Check that we can login using valid short username with very long password.""" username = "long_password" user = {"cn": username, "userpassword": "long_password_12345678901234567890123456789012345678901234567890123456789012345678901234567890"} - add_user_to_ldap_and_login(user=user, server=server) + add_user_to_ldap_and_login(user=user, server=server, rbac=rbac) @TestScenario @Requirements( RQ_SRS_007_LDAP_Authentication_Invalid("1.0") ) -def valid_short_username_and_invalid_long_password(self, server): +def valid_short_username_and_invalid_long_password(self, server, rbac=False): """Check that we can't login using valid short username and invalid long password.""" username = "long_password" user = {"cn": username, "userpassword": "long_password_12345678901234567890123456789012345678901234567890123456789012345678901234567890"} @@ -385,13 +388,13 @@ def valid_short_username_and_invalid_long_password(self, server): exitcode = 4 message=f"DB::Exception: {username}: Authentication failed: password is incorrect or there is no user with such name" - add_user_to_ldap_and_login(user=user, login=login, exitcode=exitcode, message=message, server=server) + add_user_to_ldap_and_login(user=user, login=login, exitcode=exitcode, message=message, server=server, rbac=rbac) @TestScenario @Requirements( RQ_SRS_007_LDAP_Authentication_Invalid("1.0") ) -def valid_username_and_invalid_password(self, server): +def valid_username_and_invalid_password(self, server, rbac=False): """Check that we can't login using valid username and invalid password.""" username = "valid_username_and_invalid_password" user = {"cn": username, "userpassword": username} @@ -400,13 +403,13 @@ def valid_username_and_invalid_password(self, server): exitcode = 4 message=f"DB::Exception: {username}: Authentication failed: password is incorrect or there is no user with such name" - add_user_to_ldap_and_login(user=user, login=login, exitcode=exitcode, message=message, server=server) + add_user_to_ldap_and_login(user=user, login=login, exitcode=exitcode, message=message, server=server, rbac=rbac) @TestScenario @Requirements( RQ_SRS_007_LDAP_Authentication_Invalid("1.0") ) -def invalid_username_and_valid_password(self, server): +def invalid_username_and_valid_password(self, server, rbac=False): """Check that we can't login using slightly invalid username but valid password.""" username = "invalid_username_and_valid_password" user = {"cn": username, "userpassword": username} @@ -415,7 +418,7 @@ def invalid_username_and_valid_password(self, server): exitcode = 4 message=f"DB::Exception: {login['username']}: Authentication failed: password is incorrect or there is no user with such name" - add_user_to_ldap_and_login(user=user, login=login, exitcode=exitcode, message=message, server=server) + add_user_to_ldap_and_login(user=user, login=login, exitcode=exitcode, message=message, server=server, rbac=rbac) @TestScenario @Requirements( @@ -423,37 +426,44 @@ def invalid_username_and_valid_password(self, server): RQ_SRS_007_LDAP_Authentication_Username_UTF8("1.0"), RQ_SRS_007_LDAP_Configuration_User_Name_UTF8("1.0") ) -def valid_utf8_username_and_ascii_password(self, server): +def valid_utf8_username_and_ascii_password(self, server, rbac=False): """Check that we can login using valid utf-8 username with ascii password.""" username = "utf8_username_Gãńdåłf_Thê_Gręât" user = {"cn": username, "userpassword": "utf8_username"} - add_user_to_ldap_and_login(user=user, server=server) + add_user_to_ldap_and_login(user=user, server=server, rbac=rbac) @TestScenario @Requirements( RQ_SRS_007_LDAP_Authentication_Valid("1.0"), RQ_SRS_007_LDAP_Authentication_Password_UTF8("1.0") ) -def valid_ascii_username_and_utf8_password(self, server): +def valid_ascii_username_and_utf8_password(self, server, rbac=False): """Check that we can login using valid ascii username with utf-8 password.""" username = "utf8_password" user = {"cn": username, "userpassword": "utf8_password_Gãńdåłf_Thê_Gręât"} - add_user_to_ldap_and_login(user=user, server=server) + add_user_to_ldap_and_login(user=user, server=server, rbac=rbac) @TestScenario -def empty_username_and_empty_password(self, server=None): +def empty_username_and_empty_password(self, server=None, rbac=False): """Check that we can login using empty username and empty password as it will use the default user and that has an empty password.""" login_and_execute_query(username="", password="") -@TestFeature +@TestOutline(Feature) @Name("user authentications") @Requirements( RQ_SRS_007_LDAP_Authentication_Mechanism_NamePassword("1.0") ) -def feature(self, servers=None, node="clickhouse1"): +@Examples("rbac", [ + (False,), + (True,) +]) +def feature(self, rbac, servers=None, node="clickhouse1"): + """Check that users can be authenticated using an LDAP server when + users are configured either using an XML configuration file or RBAC. + """ self.context.node = self.context.cluster.node(node) if servers is None: @@ -461,4 +471,4 @@ def feature(self, servers=None, node="clickhouse1"): with ldap_servers(servers): for scenario in loads(current_module(), Scenario): - scenario(server="openldap1") + scenario(server="openldap1", rbac=rbac) diff --git a/tests/testflows/ldap/tests/common.py b/tests/testflows/ldap/tests/common.py index a1f823550f3..0d04818131e 100644 --- a/tests/testflows/ldap/tests/common.py +++ b/tests/testflows/ldap/tests/common.py @@ -171,16 +171,35 @@ def create_ldap_users_config_content(*users, config_d_dir="/etc/clickhouse-serve return Config(content, path, name, uid, "users.xml") +def add_users_identified_with_ldap(*users): + """Add one or more users that are identified via + an ldap server using RBAC. + """ + node = current().context.node + try: + with Given("I create users"): + for user in users: + node.query(f"CREATE USER {user['username']} IDENTIFIED WITH ldap_server BY '{user['server']}'") + yield + finally: + with Finally("I remove users"): + for user in users: + with By(f"dropping user {user['username']}", flags=TE): + node.query(f"DROP USER IF EXISTS {user['username']}") + @contextmanager def ldap_authenticated_users(*users, config_d_dir="/etc/clickhouse-server/users.d", - config_file=None, timeout=20, restart=True, config=None): - """Add LDAP authenticated user configuration. + config_file=None, timeout=20, restart=True, config=None, rbac=False): + """Add LDAP authenticated users. """ - if config_file is None: - config_file = f"ldap_users_{getuid()}.xml" - if config is None: - config = create_ldap_users_config_content(*users, config_d_dir=config_d_dir, config_file=config_file) - return add_config(config, restart=restart) + if rbac: + return add_users_identified_with_ldap(*users) + else: + if config_file is None: + config_file = f"ldap_users_{getuid()}.xml" + if config is None: + config = create_ldap_users_config_content(*users, config_d_dir=config_d_dir, config_file=config_file) + return add_config(config, restart=restart) def invalid_server_config(servers, message=None, tail=13, timeout=20): """Check that ClickHouse errors when trying to load invalid LDAP servers configuration file. @@ -375,4 +394,3 @@ def login(servers, *users, config=None): settings=[("user", user["username"]), ("password", user["password"])], exitcode=user.get("exitcode", None), message=user.get("message", None)) - From a6ff049eec318773620827cbdbeb97195804ff49 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov <avtokmakov@yandex-team.ru> Date: Wed, 12 Aug 2020 23:40:13 +0300 Subject: [PATCH 297/374] use Atomic for system database --- programs/server/Server.cpp | 3 + src/Core/Settings.h | 4 +- src/Interpreters/InterpreterCreateQuery.cpp | 4 - src/Interpreters/loadMetadata.cpp | 21 +-- src/Storages/System/IStorageSystemOneBlock.h | 2 +- .../StorageSystemAsynchronousMetrics.cpp | 4 +- .../System/StorageSystemAsynchronousMetrics.h | 2 +- src/Storages/System/StorageSystemColumns.cpp | 4 +- src/Storages/System/StorageSystemColumns.h | 2 +- .../System/StorageSystemDetachedParts.cpp | 6 +- src/Storages/System/StorageSystemDisks.cpp | 4 +- src/Storages/System/StorageSystemDisks.h | 2 +- src/Storages/System/StorageSystemOne.cpp | 4 +- src/Storages/System/StorageSystemOne.h | 2 +- src/Storages/System/StorageSystemParts.cpp | 4 +- src/Storages/System/StorageSystemParts.h | 2 +- .../System/StorageSystemPartsBase.cpp | 4 +- src/Storages/System/StorageSystemPartsBase.h | 2 +- .../System/StorageSystemPartsColumns.cpp | 4 +- .../System/StorageSystemPartsColumns.h | 2 +- src/Storages/System/StorageSystemReplicas.cpp | 4 +- src/Storages/System/StorageSystemReplicas.h | 2 +- .../System/StorageSystemStackTrace.cpp | 4 +- src/Storages/System/StorageSystemStackTrace.h | 2 +- .../System/StorageSystemStoragePolicies.cpp | 4 +- .../System/StorageSystemStoragePolicies.h | 2 +- src/Storages/System/StorageSystemTables.cpp | 4 +- src/Storages/System/StorageSystemTables.h | 2 +- src/Storages/System/attachSystemTables.cpp | 138 +++++++++++------- tests/config/database_atomic_usersd.xml | 1 - .../01107_atomic_db_detach_attach.sh | 4 +- .../0_stateless/01109_exchange_tables.sql | 1 - .../0_stateless/01114_database_atomic.sh | 4 +- .../0_stateless/01192_rename_database.sh | 4 +- 34 files changed, 138 insertions(+), 120 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ddc5ec080fb..fdefd217e61 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -468,6 +468,9 @@ int Server::main(const std::vector<std::string> & /*args*/) } { + Poco::File(path + "data/").createDirectories(); + Poco::File(path + "metadata/").createDirectories(); + /// Directory with metadata of tables, which was marked as dropped by Atomic database Poco::File(path + "metadata_dropped/").createDirectories(); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 60a3792ea87..54775c79b14 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -347,7 +347,6 @@ class IColumn; M(UInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \ \ M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \ - M(Bool, allow_experimental_database_atomic, true, "Allow to create database with Engine=Atomic.", 0) \ M(Bool, show_table_uuid_in_table_create_query_if_not_nil, false, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \ M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ @@ -395,7 +394,8 @@ class IColumn; M(UInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \ \ M(Bool, force_optimize_skip_unused_shards_no_nested, false, "Obsolete setting, does nothing. Will be removed after 2020-12-01. Use force_optimize_skip_unused_shards_nesting instead.", 0) \ - M(Bool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) + M(Bool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) \ + M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing. Will be removed after 2021-02-12", 0) #define FORMAT_FACTORY_SETTINGS(M) \ M(Char, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 3f8ff0d41f2..e0e19ae317d 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -142,10 +142,6 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) if (create.storage->engine->name == "Atomic") { - if (!context.getSettingsRef().allow_experimental_database_atomic && !internal) - throw Exception("Atomic is an experimental database engine. " - "Enable allow_experimental_database_atomic to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); - if (create.attach && create.uuid == UUIDHelpers::Nil) throw Exception("UUID must be specified for ATTACH", ErrorCodes::INCORRECT_QUERY); else if (create.uuid == UUIDHelpers::Nil) diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index d7c2dfa5287..e273d6a7be1 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -84,9 +84,6 @@ static void loadDatabase( } -#define SYSTEM_DATABASE "system" - - void loadMetadata(Context & context, const String & default_database_name) { Poco::Logger * log = &Poco::Logger::get("loadMetadata"); @@ -114,7 +111,7 @@ void loadMetadata(Context & context, const String & default_database_name) if (endsWith(it.name(), ".sql")) { String db_name = it.name().substr(0, it.name().size() - 4); - if (db_name != SYSTEM_DATABASE) + if (db_name != DatabaseCatalog::SYSTEM_DATABASE) databases.emplace(unescapeForFileName(db_name), path + "/" + db_name); } @@ -140,7 +137,7 @@ void loadMetadata(Context & context, const String & default_database_name) if (it.name().at(0) == '.') continue; - if (it.name() == SYSTEM_DATABASE) + if (it.name() == DatabaseCatalog::SYSTEM_DATABASE) continue; databases.emplace(unescapeForFileName(it.name()), it.path().toString()); @@ -172,21 +169,19 @@ void loadMetadata(Context & context, const String & default_database_name) void loadMetadataSystem(Context & context) { - String path = context.getPath() + "metadata/" SYSTEM_DATABASE; + String path = context.getPath() + "metadata/" + DatabaseCatalog::SYSTEM_DATABASE; if (Poco::File(path).exists()) { /// 'has_force_restore_data_flag' is true, to not fail on loading query_log table, if it is corrupted. - loadDatabase(context, SYSTEM_DATABASE, path, true); + loadDatabase(context, DatabaseCatalog::SYSTEM_DATABASE, path, true); } else { /// Initialize system database manually - String global_path = context.getPath(); - Poco::File(global_path + "data/" SYSTEM_DATABASE).createDirectories(); - Poco::File(global_path + "metadata/" SYSTEM_DATABASE).createDirectories(); - - auto system_database = std::make_shared<DatabaseOrdinary>(SYSTEM_DATABASE, global_path + "metadata/" SYSTEM_DATABASE "/", context); - DatabaseCatalog::instance().attachDatabase(SYSTEM_DATABASE, system_database); + String database_create_query = "CREATE DATABASE "; + database_create_query += DatabaseCatalog::SYSTEM_DATABASE; + database_create_query += " ENGINE=Atomic"; + executeCreateQuery(database_create_query, context, DatabaseCatalog::SYSTEM_DATABASE, "<no file>", true); } } diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 7c2ef85f158..c2abb5302fa 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -21,7 +21,7 @@ protected: virtual void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const = 0; public: - IStorageSystemOneBlock(const String & name_) : IStorage({"system", name_}) + IStorageSystemOneBlock(const StorageID & table_id_) : IStorage(table_id_) { StorageInMemoryMetadata metadata_; metadata_.setColumns(ColumnsDescription(Self::getNamesAndTypes())); diff --git a/src/Storages/System/StorageSystemAsynchronousMetrics.cpp b/src/Storages/System/StorageSystemAsynchronousMetrics.cpp index 059ef708a81..8dabac4fb49 100644 --- a/src/Storages/System/StorageSystemAsynchronousMetrics.cpp +++ b/src/Storages/System/StorageSystemAsynchronousMetrics.cpp @@ -16,8 +16,8 @@ NamesAndTypesList StorageSystemAsynchronousMetrics::getNamesAndTypes() } -StorageSystemAsynchronousMetrics::StorageSystemAsynchronousMetrics(const std::string & name_, const AsynchronousMetrics & async_metrics_) - : IStorageSystemOneBlock(name_), async_metrics(async_metrics_) +StorageSystemAsynchronousMetrics::StorageSystemAsynchronousMetrics(const StorageID & table_id_, const AsynchronousMetrics & async_metrics_) + : IStorageSystemOneBlock(table_id_), async_metrics(async_metrics_) { } diff --git a/src/Storages/System/StorageSystemAsynchronousMetrics.h b/src/Storages/System/StorageSystemAsynchronousMetrics.h index f31450fe086..8f79c09b098 100644 --- a/src/Storages/System/StorageSystemAsynchronousMetrics.h +++ b/src/Storages/System/StorageSystemAsynchronousMetrics.h @@ -25,7 +25,7 @@ private: const AsynchronousMetrics & async_metrics; protected: - StorageSystemAsynchronousMetrics(const std::string & name_, const AsynchronousMetrics & async_metrics_); + StorageSystemAsynchronousMetrics(const StorageID & table_id_, const AsynchronousMetrics & async_metrics_); void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 57d8d13e79b..3c6bb493830 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -23,8 +23,8 @@ namespace ErrorCodes extern const int TABLE_IS_DROPPED; } -StorageSystemColumns::StorageSystemColumns(const std::string & name_) - : IStorage({"system", name_}) +StorageSystemColumns::StorageSystemColumns(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription( diff --git a/src/Storages/System/StorageSystemColumns.h b/src/Storages/System/StorageSystemColumns.h index 7336b406183..89b33214352 100644 --- a/src/Storages/System/StorageSystemColumns.h +++ b/src/Storages/System/StorageSystemColumns.h @@ -27,7 +27,7 @@ public: unsigned num_streams) override; protected: - StorageSystemColumns(const std::string & name_); + StorageSystemColumns(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index c325df1251d..b0953ac75e1 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -27,8 +27,8 @@ public: std::string getName() const override { return "SystemDetachedParts"; } protected: - explicit StorageSystemDetachedParts() - : IStorage({"system", "detached_parts"}) + explicit StorageSystemDetachedParts(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription{{ @@ -90,7 +90,7 @@ protected: StoragePtr createDetachedPartsTable() { - return StorageSystemDetachedParts::create(); + return StorageSystemDetachedParts::create(StorageID{"system", "detached_parts"}); } } diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index 554b8cfd1eb..d4ace6378d9 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -11,8 +11,8 @@ namespace ErrorCodes } -StorageSystemDisks::StorageSystemDisks(const std::string & name_) - : IStorage({"system", name_}) +StorageSystemDisks::StorageSystemDisks(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription( diff --git a/src/Storages/System/StorageSystemDisks.h b/src/Storages/System/StorageSystemDisks.h index 714a0a5428c..b5968556351 100644 --- a/src/Storages/System/StorageSystemDisks.h +++ b/src/Storages/System/StorageSystemDisks.h @@ -30,7 +30,7 @@ public: unsigned num_streams) override; protected: - StorageSystemDisks(const std::string & name_); + StorageSystemDisks(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemOne.cpp b/src/Storages/System/StorageSystemOne.cpp index 3329cbb035e..b29051a796d 100644 --- a/src/Storages/System/StorageSystemOne.cpp +++ b/src/Storages/System/StorageSystemOne.cpp @@ -11,8 +11,8 @@ namespace DB { -StorageSystemOne::StorageSystemOne(const std::string & name_) - : IStorage({"system", name_}) +StorageSystemOne::StorageSystemOne(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription({{"dummy", std::make_shared<DataTypeUInt8>()}})); diff --git a/src/Storages/System/StorageSystemOne.h b/src/Storages/System/StorageSystemOne.h index 3b3e531dc86..044fb39a1ee 100644 --- a/src/Storages/System/StorageSystemOne.h +++ b/src/Storages/System/StorageSystemOne.h @@ -31,7 +31,7 @@ public: unsigned num_streams) override; protected: - StorageSystemOne(const std::string & name_); + StorageSystemOne(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 7c0da07ed88..a3fb2cf10e6 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -15,8 +15,8 @@ namespace DB { -StorageSystemParts::StorageSystemParts(const std::string & name_) - : StorageSystemPartsBase(name_, +StorageSystemParts::StorageSystemParts(const StorageID & table_id_) + : StorageSystemPartsBase(table_id_, { {"partition", std::make_shared<DataTypeString>()}, {"name", std::make_shared<DataTypeString>()}, diff --git a/src/Storages/System/StorageSystemParts.h b/src/Storages/System/StorageSystemParts.h index 06f4f414c88..2de4cd3f3a4 100644 --- a/src/Storages/System/StorageSystemParts.h +++ b/src/Storages/System/StorageSystemParts.h @@ -19,7 +19,7 @@ public: std::string getName() const override { return "SystemParts"; } protected: - explicit StorageSystemParts(const std::string & name_); + explicit StorageSystemParts(const StorageID & table_id_); void processNextStorage(MutableColumns & columns, const StoragesInfo & info, bool has_state_column) override; }; diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 168b305605d..4b98089f95e 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -261,8 +261,8 @@ Pipes StorageSystemPartsBase::read( } -StorageSystemPartsBase::StorageSystemPartsBase(std::string name_, NamesAndTypesList && columns_) - : IStorage(StorageID{"system", name_}) +StorageSystemPartsBase::StorageSystemPartsBase(const StorageID & table_id_, NamesAndTypesList && columns_) + : IStorage(table_id_) { ColumnsDescription tmp_columns(std::move(columns_)); diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 56c9a8fb0d0..ff338930e0c 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -72,7 +72,7 @@ private: protected: const FormatSettings format_settings; - StorageSystemPartsBase(std::string name_, NamesAndTypesList && columns_); + StorageSystemPartsBase(const StorageID & table_id_, NamesAndTypesList && columns_); virtual void processNextStorage(MutableColumns & columns, const StoragesInfo & info, bool has_state_column) = 0; }; diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index 4631bb9c4c5..0851e389e4f 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -15,8 +15,8 @@ namespace DB { -StorageSystemPartsColumns::StorageSystemPartsColumns(const std::string & name_) - : StorageSystemPartsBase(name_, +StorageSystemPartsColumns::StorageSystemPartsColumns(const StorageID & table_id_) + : StorageSystemPartsBase(table_id_, { {"partition", std::make_shared<DataTypeString>()}, {"name", std::make_shared<DataTypeString>()}, diff --git a/src/Storages/System/StorageSystemPartsColumns.h b/src/Storages/System/StorageSystemPartsColumns.h index 596fd7dae45..6347a418875 100644 --- a/src/Storages/System/StorageSystemPartsColumns.h +++ b/src/Storages/System/StorageSystemPartsColumns.h @@ -21,7 +21,7 @@ public: std::string getName() const override { return "SystemPartsColumns"; } protected: - StorageSystemPartsColumns(const std::string & name_); + StorageSystemPartsColumns(const StorageID & table_id_); void processNextStorage(MutableColumns & columns, const StoragesInfo & info, bool has_state_column) override; }; diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 27a9cd0c4bb..211859b27ff 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -16,8 +16,8 @@ namespace DB { -StorageSystemReplicas::StorageSystemReplicas(const std::string & name_) - : IStorage({"system", name_}) +StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription({ diff --git a/src/Storages/System/StorageSystemReplicas.h b/src/Storages/System/StorageSystemReplicas.h index b068ebc8b0a..198a7afa0fe 100644 --- a/src/Storages/System/StorageSystemReplicas.h +++ b/src/Storages/System/StorageSystemReplicas.h @@ -28,7 +28,7 @@ public: unsigned num_streams) override; protected: - StorageSystemReplicas(const std::string & name_); + StorageSystemReplicas(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index bd9915237d4..9edcb1ede47 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -126,8 +126,8 @@ namespace } -StorageSystemStackTrace::StorageSystemStackTrace(const String & name_) - : IStorageSystemOneBlock<StorageSystemStackTrace>(name_) +StorageSystemStackTrace::StorageSystemStackTrace(const StorageID & table_id_) + : IStorageSystemOneBlock<StorageSystemStackTrace>(table_id_) { notification_pipe.open(); diff --git a/src/Storages/System/StorageSystemStackTrace.h b/src/Storages/System/StorageSystemStackTrace.h index 3f20b1e973c..a389f02eb09 100644 --- a/src/Storages/System/StorageSystemStackTrace.h +++ b/src/Storages/System/StorageSystemStackTrace.h @@ -23,7 +23,7 @@ public: String getName() const override { return "SystemStackTrace"; } static NamesAndTypesList getNamesAndTypes(); - StorageSystemStackTrace(const String & name_); + StorageSystemStackTrace(const StorageID & table_id_); protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemStoragePolicies.cpp b/src/Storages/System/StorageSystemStoragePolicies.cpp index 76a665d871c..788ee05ec95 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.cpp +++ b/src/Storages/System/StorageSystemStoragePolicies.cpp @@ -17,8 +17,8 @@ namespace ErrorCodes } -StorageSystemStoragePolicies::StorageSystemStoragePolicies(const std::string & name_) - : IStorage({"system", name_}) +StorageSystemStoragePolicies::StorageSystemStoragePolicies(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns( diff --git a/src/Storages/System/StorageSystemStoragePolicies.h b/src/Storages/System/StorageSystemStoragePolicies.h index a1427da8559..a22563c9543 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.h +++ b/src/Storages/System/StorageSystemStoragePolicies.h @@ -30,7 +30,7 @@ public: unsigned num_streams) override; protected: - StorageSystemStoragePolicies(const std::string & name_); + StorageSystemStoragePolicies(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index b8a65183be1..2974e3f493d 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -30,8 +30,8 @@ namespace ErrorCodes } -StorageSystemTables::StorageSystemTables(const std::string & name_) - : IStorage({"system", name_}) +StorageSystemTables::StorageSystemTables(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription( diff --git a/src/Storages/System/StorageSystemTables.h b/src/Storages/System/StorageSystemTables.h index 54551205684..354d87adf1c 100644 --- a/src/Storages/System/StorageSystemTables.h +++ b/src/Storages/System/StorageSystemTables.h @@ -28,7 +28,7 @@ public: unsigned num_streams) override; protected: - StorageSystemTables(const std::string & name_); + StorageSystemTables(const StorageID & table_id_); }; } diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 2b52f0fe5cc..89c8de7f430 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -66,79 +66,105 @@ namespace DB { +namespace +{ + +template<typename StorageT> +void attach(IDatabase & system_database, const String & table_name) +{ + if (system_database.getUUID() == UUIDHelpers::Nil) + { + auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name); + system_database.attachTable(table_name, StorageT::create(table_id)); + } + else + { + auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name, UUIDHelpers::generateV4()); + String path = "store/" + DatabaseCatalog::getPathForUUID(table_id.uuid); + system_database.attachTable(table_name, StorageT::create(table_id), path); + } +} + +} + + ///TODO allow store system tables in DatabaseAtomic void attachSystemTablesLocal(IDatabase & system_database) { - system_database.attachTable("one", StorageSystemOne::create("one")); - system_database.attachTable("numbers", StorageSystemNumbers::create(StorageID("system", "numbers"), false)); - system_database.attachTable("numbers_mt", StorageSystemNumbers::create(StorageID("system", "numbers_mt"), true)); - system_database.attachTable("zeros", StorageSystemZeros::create(StorageID("system", "zeros"), false)); - system_database.attachTable("zeros_mt", StorageSystemZeros::create(StorageID("system", "zeros_mt"), true)); - system_database.attachTable("databases", StorageSystemDatabases::create("databases")); - system_database.attachTable("tables", StorageSystemTables::create("tables")); - system_database.attachTable("columns", StorageSystemColumns::create("columns")); - system_database.attachTable("functions", StorageSystemFunctions::create("functions")); - system_database.attachTable("events", StorageSystemEvents::create("events")); - system_database.attachTable("settings", StorageSystemSettings::create("settings")); - system_database.attachTable("merge_tree_settings", SystemMergeTreeSettings::create("merge_tree_settings")); - system_database.attachTable("build_options", StorageSystemBuildOptions::create("build_options")); - system_database.attachTable("formats", StorageSystemFormats::create("formats")); - system_database.attachTable("table_functions", StorageSystemTableFunctions::create("table_functions")); - system_database.attachTable("aggregate_function_combinators", StorageSystemAggregateFunctionCombinators::create("aggregate_function_combinators")); - system_database.attachTable("data_type_families", StorageSystemDataTypeFamilies::create("data_type_families")); - system_database.attachTable("collations", StorageSystemCollations::create("collations")); - system_database.attachTable("table_engines", StorageSystemTableEngines::create("table_engines")); - system_database.attachTable("contributors", StorageSystemContributors::create("contributors")); - system_database.attachTable("users", StorageSystemUsers::create("users")); - system_database.attachTable("roles", StorageSystemRoles::create("roles")); - system_database.attachTable("grants", StorageSystemGrants::create("grants")); - system_database.attachTable("role_grants", StorageSystemRoleGrants::create("role_grants")); - system_database.attachTable("current_roles", StorageSystemCurrentRoles::create("current_roles")); - system_database.attachTable("enabled_roles", StorageSystemEnabledRoles::create("enabled_roles")); - system_database.attachTable("settings_profiles", StorageSystemSettingsProfiles::create("settings_profiles")); - system_database.attachTable("settings_profile_elements", StorageSystemSettingsProfileElements::create("settings_profile_elements")); - system_database.attachTable("row_policies", StorageSystemRowPolicies::create("row_policies")); - system_database.attachTable("quotas", StorageSystemQuotas::create("quotas")); - system_database.attachTable("quota_limits", StorageSystemQuotaLimits::create("quota_limits")); - system_database.attachTable("quota_usage", StorageSystemQuotaUsage::create("quota_usage")); - system_database.attachTable("quotas_usage", StorageSystemQuotasUsage::create("all_quotas_usage")); - system_database.attachTable("privileges", StorageSystemPrivileges::create("privileges")); + attach<StorageSystemOne>(system_database, "one"); + + //system_database.attachTable("numbers", StorageSystemNumbers::create(StorageID("system", "numbers"), false)); + //system_database.attachTable("numbers_mt", StorageSystemNumbers::create(StorageID("system", "numbers_mt"), true)); + //system_database.attachTable("zeros", StorageSystemZeros::create(StorageID("system", "zeros"), false)); + //system_database.attachTable("zeros_mt", StorageSystemZeros::create(StorageID("system", "zeros_mt"), true)); + + attach<StorageSystemDatabases>(system_database, "databases"); + attach<StorageSystemTables>(system_database, "tables"); + attach<StorageSystemColumns>(system_database, "columns"); + attach<StorageSystemFunctions>(system_database, "functions"); + attach<StorageSystemEvents>(system_database, "events"); + attach<StorageSystemSettings>(system_database, "settings"); + attach<SystemMergeTreeSettings>(system_database, "merge_tree_settings"); + attach<StorageSystemBuildOptions>(system_database, "build_options"); + attach<StorageSystemFormats>(system_database, "formats"); + attach<StorageSystemTableFunctions>(system_database, "table_functions"); + attach<StorageSystemAggregateFunctionCombinators>(system_database, "aggregate_function_combinators"); + attach<StorageSystemDataTypeFamilies>(system_database, "data_type_families"); + attach<StorageSystemCollations>(system_database, "collations"); + attach<StorageSystemTableEngines>(system_database, "table_engines"); + attach<StorageSystemContributors>(system_database, "contributors"); + attach<StorageSystemUsers>(system_database, "users"); + attach<StorageSystemRoles>(system_database, "roles"); + attach<StorageSystemGrants>(system_database, "grants"); + attach<StorageSystemRoleGrants>(system_database, "role_grants"); + attach<StorageSystemCurrentRoles>(system_database, "current_roles"); + attach<StorageSystemEnabledRoles>(system_database, "enabled_roles"); + attach<StorageSystemSettingsProfiles>(system_database, "settings_profiles"); + attach<StorageSystemSettingsProfileElements>(system_database, "settings_profile_elements"); + attach<StorageSystemRowPolicies>(system_database, "row_policies"); + attach<StorageSystemQuotas>(system_database, "quotas"); + attach<StorageSystemQuotaLimits>(system_database, "quota_limits"); + attach<StorageSystemQuotaUsage>(system_database, "quota_usage"); + attach<StorageSystemQuotasUsage>(system_database, "quotas_usage"); + attach<StorageSystemPrivileges>(system_database, "privileges"); + #if !defined(ARCADIA_BUILD) - system_database.attachTable("licenses", StorageSystemLicenses::create("licenses")); + attach<StorageSystemLicenses>(system_database, "licenses"); #endif #ifdef OS_LINUX - system_database.attachTable("stack_trace", StorageSystemStackTrace::create("stack_trace")); + attach<StorageSystemStackTrace>(system_database, "stack_trace"); #endif } void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper) { attachSystemTablesLocal(system_database); - system_database.attachTable("parts", StorageSystemParts::create("parts")); - system_database.attachTable("detached_parts", createDetachedPartsTable()); - system_database.attachTable("parts_columns", StorageSystemPartsColumns::create("parts_columns")); - system_database.attachTable("disks", StorageSystemDisks::create("disks")); - system_database.attachTable("storage_policies", StorageSystemStoragePolicies::create("storage_policies")); - system_database.attachTable("processes", StorageSystemProcesses::create("processes")); - system_database.attachTable("metrics", StorageSystemMetrics::create("metrics")); - system_database.attachTable("merges", StorageSystemMerges::create("merges")); - system_database.attachTable("mutations", StorageSystemMutations::create("mutations")); - system_database.attachTable("replicas", StorageSystemReplicas::create("replicas")); - system_database.attachTable("replication_queue", StorageSystemReplicationQueue::create("replication_queue")); - system_database.attachTable("distribution_queue", StorageSystemDistributionQueue::create("distribution_queue")); - system_database.attachTable("dictionaries", StorageSystemDictionaries::create("dictionaries")); - system_database.attachTable("models", StorageSystemModels::create("models")); - system_database.attachTable("clusters", StorageSystemClusters::create("clusters")); - system_database.attachTable("graphite_retentions", StorageSystemGraphite::create("graphite_retentions")); - system_database.attachTable("macros", StorageSystemMacros::create("macros")); + + attach<StorageSystemParts>(system_database, "parts"); + //attach<>(system_database, "detached_parts", createDetachedPartsTable()); + attach<StorageSystemPartsColumns>(system_database, "parts_columns"); + attach<StorageSystemDisks>(system_database, "disks"); + attach<StorageSystemStoragePolicies>(system_database, "storage_policies"); + attach<StorageSystemProcesses>(system_database, "processes"); + attach<StorageSystemMetrics>(system_database, "metrics"); + attach<StorageSystemMerges>(system_database, "merges"); + attach<StorageSystemMutations>(system_database, "mutations"); + attach<StorageSystemReplicas>(system_database, "replicas"); + attach<StorageSystemReplicationQueue>(system_database, "replication_queue"); + attach<StorageSystemDistributionQueue>(system_database, "distribution_queue"); + attach<StorageSystemDictionaries>(system_database, "dictionaries"); + attach<StorageSystemModels>(system_database, "models"); + attach<StorageSystemClusters>(system_database, "clusters"); + attach<StorageSystemGraphite>(system_database, "graphite_retentions"); + attach<StorageSystemMacros>(system_database, "macros"); if (has_zookeeper) - system_database.attachTable("zookeeper", StorageSystemZooKeeper::create("zookeeper")); + attach<StorageSystemZooKeeper>(system_database, "zookeeper"); } -void attachSystemTablesAsync(IDatabase & system_database, AsynchronousMetrics & async_metrics) +void attachSystemTablesAsync(IDatabase & /*system_database*/, AsynchronousMetrics & /*async_metrics*/) { - system_database.attachTable("asynchronous_metrics", StorageSystemAsynchronousMetrics::create("asynchronous_metrics", async_metrics)); + //system_database.attachTable("asynchronous_metrics", StorageSystemAsynchronousMetrics::create("asynchronous_metrics", async_metrics)); } } diff --git a/tests/config/database_atomic_usersd.xml b/tests/config/database_atomic_usersd.xml index 9e749b231bd..201d476da24 100644 --- a/tests/config/database_atomic_usersd.xml +++ b/tests/config/database_atomic_usersd.xml @@ -2,7 +2,6 @@ <profiles> <default> <default_database_engine>Atomic</default_database_engine> - <allow_experimental_database_atomic>1</allow_experimental_database_atomic> <show_table_uuid_in_table_create_query_if_not_nil>0</show_table_uuid_in_table_create_query_if_not_nil> </default> </profiles> diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh index 18d74a1817c..9f4bfb2c436 100755 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01107" -$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01107 ENGINE=Atomic" +$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01107 ENGINE=Atomic" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01107.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" $CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & @@ -18,7 +18,7 @@ wait $CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" $CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" -$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 -q "ATTACH DATABASE test_01107" +$CLICKHOUSE_CLIENT -q "ATTACH DATABASE test_01107" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" $CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" & diff --git a/tests/queries/0_stateless/01109_exchange_tables.sql b/tests/queries/0_stateless/01109_exchange_tables.sql index 7125bfea851..7fbb36e8ce9 100644 --- a/tests/queries/0_stateless/01109_exchange_tables.sql +++ b/tests/queries/0_stateless/01109_exchange_tables.sql @@ -1,5 +1,4 @@ DROP DATABASE IF EXISTS test_01109; -SET allow_experimental_database_atomic=1; CREATE DATABASE test_01109 ENGINE=Atomic; USE test_01109; diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index c7862bf5314..d23be44e784 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -8,8 +8,8 @@ $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_2" $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_3" -$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01114_1 ENGINE=Atomic" -$CLICKHOUSE_CLIENT --default_database_engine=Atomic --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01114_2" +$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Atomic" +$CLICKHOUSE_CLIENT --default_database_engine=Atomic -q "CREATE DATABASE test_01114_2" $CLICKHOUSE_CLIENT --default_database_engine=Ordinary -q "CREATE DATABASE test_01114_3" $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE test_01114_1" diff --git a/tests/queries/0_stateless/01192_rename_database.sh b/tests/queries/0_stateless/01192_rename_database.sh index 3d4223d09d5..c97ec14335c 100755 --- a/tests/queries/0_stateless/01192_rename_database.sh +++ b/tests/queries/0_stateless/01192_rename_database.sh @@ -10,7 +10,7 @@ $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01192_renamed" $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01192_atomic" $CLICKHOUSE_CLIENT --default_database_engine=Ordinary -q "CREATE DATABASE test_01192 UUID '00001192-0000-4000-8000-000000000001'" 2>&1| grep -F "does not support" > /dev/null && echo "ok" -$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 --default_database_engine=Atomic -q "CREATE DATABASE test_01192 UUID '00001192-0000-4000-8000-000000000001'" +$CLICKHOUSE_CLIENT --default_database_engine=Atomic -q "CREATE DATABASE test_01192 UUID '00001192-0000-4000-8000-000000000001'" # 2. check metadata $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE DATABASE test_01192" @@ -40,7 +40,7 @@ $CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW test_01192.mv TO test_01192.rmt $CLICKHOUSE_CLIENT -q "INSERT INTO test_01192.mt SELECT number FROM numbers(10)" && echo "inserted" -$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 --default_database_engine=Atomic -q "CREATE DATABASE test_01192_atomic" +$CLICKHOUSE_CLIENT --default_database_engine=Atomic -q "CREATE DATABASE test_01192_atomic" $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01192_renamed" # it's blocking $CLICKHOUSE_CLIENT -q "RENAME TABLE test_01192.mt TO test_01192_atomic.mt, test_01192.rmt TO test_01192_atomic.rmt, test_01192.mv TO test_01192_atomic.mv" && echo "renamed" From 034c680a05be8225e838dcd130b3fa6ae839ea3b Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov <vzakaznikov@protonmail.com> Date: Wed, 12 Aug 2020 16:55:46 -0400 Subject: [PATCH 298/374] * Adding Markdown source for the requirements * Adding a requirement to be able to add users identified with an LDAP server using RBAC * Updating requirements.py --- .../ldap/requirements/requirements.md | 544 ++++++++++++++++++ .../ldap/requirements/requirements.py | 64 ++- tests/testflows/ldap/tests/authentications.py | 2 +- 3 files changed, 586 insertions(+), 24 deletions(-) create mode 100644 tests/testflows/ldap/requirements/requirements.md diff --git a/tests/testflows/ldap/requirements/requirements.md b/tests/testflows/ldap/requirements/requirements.md new file mode 100644 index 00000000000..6d787670138 --- /dev/null +++ b/tests/testflows/ldap/requirements/requirements.md @@ -0,0 +1,544 @@ +# SRS-007 ClickHouse Authentication of Users via LDAP + +## Table of Contents + +* 1 [Revision History](#revision-history) +* 2 [Introduction](#introduction) +* 3 [Terminology](#terminology) +* 4 [Requirements](#requirements) + * 4.1 [Generic](#generic) + * 4.1.1 [RQ.SRS-007.LDAP.Authentication](#rqsrs-007ldapauthentication) + * 4.1.2 [RQ.SRS-007.LDAP.Authentication.MultipleServers](#rqsrs-007ldapauthenticationmultipleservers) + * 4.1.3 [RQ.SRS-007.LDAP.Authentication.Protocol.PlainText](#rqsrs-007ldapauthenticationprotocolplaintext) + * 4.1.4 [RQ.SRS-007.LDAP.Authentication.Protocol.TLS](#rqsrs-007ldapauthenticationprotocoltls) + * 4.1.5 [RQ.SRS-007.LDAP.Authentication.Protocol.StartTLS](#rqsrs-007ldapauthenticationprotocolstarttls) + * 4.1.6 [RQ.SRS-007.LDAP.Authentication.TLS.Certificate.Validation](#rqsrs-007ldapauthenticationtlscertificatevalidation) + * 4.1.7 [RQ.SRS-007.LDAP.Authentication.TLS.Certificate.SelfSigned](#rqsrs-007ldapauthenticationtlscertificateselfsigned) + * 4.1.8 [RQ.SRS-007.LDAP.Authentication.TLS.Certificate.SpecificCertificationAuthority](#rqsrs-007ldapauthenticationtlscertificatespecificcertificationauthority) + * 4.1.9 [RQ.SRS-007.LDAP.Server.Configuration.Invalid](#rqsrs-007ldapserverconfigurationinvalid) + * 4.1.10 [RQ.SRS-007.LDAP.User.Configuration.Invalid](#rqsrs-007ldapuserconfigurationinvalid) + * 4.1.11 [RQ.SRS-007.LDAP.Authentication.Mechanism.Anonymous](#rqsrs-007ldapauthenticationmechanismanonymous) + * 4.1.12 [RQ.SRS-007.LDAP.Authentication.Mechanism.Unauthenticated](#rqsrs-007ldapauthenticationmechanismunauthenticated) + * 4.1.13 [RQ.SRS-007.LDAP.Authentication.Mechanism.NamePassword](#rqsrs-007ldapauthenticationmechanismnamepassword) + * 4.1.14 [RQ.SRS-007.LDAP.Authentication.Valid](#rqsrs-007ldapauthenticationvalid) + * 4.1.15 [RQ.SRS-007.LDAP.Authentication.Invalid](#rqsrs-007ldapauthenticationinvalid) + * 4.1.16 [RQ.SRS-007.LDAP.Authentication.Invalid.DeletedUser](#rqsrs-007ldapauthenticationinvaliddeleteduser) + * 4.1.17 [RQ.SRS-007.LDAP.Authentication.UsernameChanged](#rqsrs-007ldapauthenticationusernamechanged) + * 4.1.18 [RQ.SRS-007.LDAP.Authentication.PasswordChanged](#rqsrs-007ldapauthenticationpasswordchanged) + * 4.1.19 [RQ.SRS-007.LDAP.Authentication.LDAPServerRestart](#rqsrs-007ldapauthenticationldapserverrestart) + * 4.1.20 [RQ.SRS-007.LDAP.Authentication.ClickHouseServerRestart](#rqsrs-007ldapauthenticationclickhouseserverrestart) + * 4.1.21 [RQ.SRS-007.LDAP.Authentication.Parallel](#rqsrs-007ldapauthenticationparallel) + * 4.1.22 [RQ.SRS-007.LDAP.Authentication.Parallel.ValidAndInvalid](#rqsrs-007ldapauthenticationparallelvalidandinvalid) + * 4.2 [Specific](#specific) + * 4.2.1 [RQ.SRS-007.LDAP.UnreachableServer](#rqsrs-007ldapunreachableserver) + * 4.2.2 [RQ.SRS-007.LDAP.Configuration.Server.Name](#rqsrs-007ldapconfigurationservername) + * 4.2.3 [RQ.SRS-007.LDAP.Configuration.Server.Host](#rqsrs-007ldapconfigurationserverhost) + * 4.2.4 [RQ.SRS-007.LDAP.Configuration.Server.Port](#rqsrs-007ldapconfigurationserverport) + * 4.2.5 [RQ.SRS-007.LDAP.Configuration.Server.Port.Default](#rqsrs-007ldapconfigurationserverportdefault) + * 4.2.6 [RQ.SRS-007.LDAP.Configuration.Server.AuthDN.Prefix](#rqsrs-007ldapconfigurationserverauthdnprefix) + * 4.2.7 [RQ.SRS-007.LDAP.Configuration.Server.AuthDN.Suffix](#rqsrs-007ldapconfigurationserverauthdnsuffix) + * 4.2.8 [RQ.SRS-007.LDAP.Configuration.Server.AuthDN.Value](#rqsrs-007ldapconfigurationserverauthdnvalue) + * 4.2.9 [RQ.SRS-007.LDAP.Configuration.Server.EnableTLS](#rqsrs-007ldapconfigurationserverenabletls) + * 4.2.10 [RQ.SRS-007.LDAP.Configuration.Server.EnableTLS.Options.Default](#rqsrs-007ldapconfigurationserverenabletlsoptionsdefault) + * 4.2.11 [RQ.SRS-007.LDAP.Configuration.Server.EnableTLS.Options.No](#rqsrs-007ldapconfigurationserverenabletlsoptionsno) + * 4.2.12 [RQ.SRS-007.LDAP.Configuration.Server.EnableTLS.Options.Yes](#rqsrs-007ldapconfigurationserverenabletlsoptionsyes) + * 4.2.13 [RQ.SRS-007.LDAP.Configuration.Server.EnableTLS.Options.StartTLS](#rqsrs-007ldapconfigurationserverenabletlsoptionsstarttls) + * 4.2.14 [RQ.SRS-007.LDAP.Configuration.Server.TLSMinimumProtocolVersion](#rqsrs-007ldapconfigurationservertlsminimumprotocolversion) + * 4.2.15 [RQ.SRS-007.LDAP.Configuration.Server.TLSMinimumProtocolVersion.Values](#rqsrs-007ldapconfigurationservertlsminimumprotocolversionvalues) + * 4.2.16 [RQ.SRS-007.LDAP.Configuration.Server.TLSMinimumProtocolVersion.Default](#rqsrs-007ldapconfigurationservertlsminimumprotocolversiondefault) + * 4.2.17 [RQ.SRS-007.LDAP.Configuration.Server.TLSRequireCert](#rqsrs-007ldapconfigurationservertlsrequirecert) + * 4.2.18 [RQ.SRS-007.LDAP.Configuration.Server.TLSRequireCert.Options.Default](#rqsrs-007ldapconfigurationservertlsrequirecertoptionsdefault) + * 4.2.19 [RQ.SRS-007.LDAP.Configuration.Server.TLSRequireCert.Options.Demand](#rqsrs-007ldapconfigurationservertlsrequirecertoptionsdemand) + * 4.2.20 [RQ.SRS-007.LDAP.Configuration.Server.TLSRequireCert.Options.Allow](#rqsrs-007ldapconfigurationservertlsrequirecertoptionsallow) + * 4.2.21 [RQ.SRS-007.LDAP.Configuration.Server.TLSRequireCert.Options.Try](#rqsrs-007ldapconfigurationservertlsrequirecertoptionstry) + * 4.2.22 [RQ.SRS-007.LDAP.Configuration.Server.TLSRequireCert.Options.Never](#rqsrs-007ldapconfigurationservertlsrequirecertoptionsnever) + * 4.2.23 [RQ.SRS-007.LDAP.Configuration.Server.TLSCertFile](#rqsrs-007ldapconfigurationservertlscertfile) + * 4.2.24 [RQ.SRS-007.LDAP.Configuration.Server.TLSKeyFile](#rqsrs-007ldapconfigurationservertlskeyfile) + * 4.2.25 [RQ.SRS-007.LDAP.Configuration.Server.TLSCACertDir](#rqsrs-007ldapconfigurationservertlscacertdir) + * 4.2.26 [RQ.SRS-007.LDAP.Configuration.Server.TLSCACertFile](#rqsrs-007ldapconfigurationservertlscacertfile) + * 4.2.27 [RQ.SRS-007.LDAP.Configuration.Server.TLSCipherSuite](#rqsrs-007ldapconfigurationservertlsciphersuite) + * 4.2.28 [RQ.SRS-007.LDAP.Configuration.Server.Syntax](#rqsrs-007ldapconfigurationserversyntax) + * 4.2.29 [RQ.SRS-007.LDAP.Configuration.User.RBAC](#rqsrs-007ldapconfigurationuserrbac) + * 4.2.30 [RQ.SRS-007.LDAP.Configuration.User.Syntax](#rqsrs-007ldapconfigurationusersyntax) + * 4.2.31 [RQ.SRS-007.LDAP.Configuration.User.Name.Empty](#rqsrs-007ldapconfigurationusernameempty) + * 4.2.32 [RQ.SRS-007.LDAP.Configuration.User.BothPasswordAndLDAP](#rqsrs-007ldapconfigurationuserbothpasswordandldap) + * 4.2.33 [RQ.SRS-007.LDAP.Configuration.User.LDAP.InvalidServerName.NotDefined](#rqsrs-007ldapconfigurationuserldapinvalidservernamenotdefined) + * 4.2.34 [RQ.SRS-007.LDAP.Configuration.User.LDAP.InvalidServerName.Empty](#rqsrs-007ldapconfigurationuserldapinvalidservernameempty) + * 4.2.35 [RQ.SRS-007.LDAP.Configuration.User.OnlyOneServer](#rqsrs-007ldapconfigurationuseronlyoneserver) + * 4.2.36 [RQ.SRS-007.LDAP.Configuration.User.Name.Long](#rqsrs-007ldapconfigurationusernamelong) + * 4.2.37 [RQ.SRS-007.LDAP.Configuration.User.Name.UTF8](#rqsrs-007ldapconfigurationusernameutf8) + * 4.2.38 [RQ.SRS-007.LDAP.Authentication.Username.Empty](#rqsrs-007ldapauthenticationusernameempty) + * 4.2.39 [RQ.SRS-007.LDAP.Authentication.Username.Long](#rqsrs-007ldapauthenticationusernamelong) + * 4.2.40 [RQ.SRS-007.LDAP.Authentication.Username.UTF8](#rqsrs-007ldapauthenticationusernameutf8) + * 4.2.41 [RQ.SRS-007.LDAP.Authentication.Password.Empty](#rqsrs-007ldapauthenticationpasswordempty) + * 4.2.42 [RQ.SRS-007.LDAP.Authentication.Password.Long](#rqsrs-007ldapauthenticationpasswordlong) + * 4.2.43 [RQ.SRS-007.LDAP.Authentication.Password.UTF8](#rqsrs-007ldapauthenticationpasswordutf8) +* 5 [References](#references) + +## Revision History + +This document is stored in an electronic form using [Git] source control management software +hosted in a [GitHub Repository]. +All the updates are tracked using the [Git]'s [Revision History]. + +## Introduction + +[ClickHouse] currently does not have any integration with [LDAP]. +As the initial step in integrating with [LDAP] this software requirements specification covers +only the requirements to enable authentication of users using an [LDAP] server. + +## Terminology + +* **CA** - + Certificate Authority ([CA]) + +* **LDAP** - + Lightweight Directory Access Protocol ([LDAP]) + +## Requirements + +### Generic + +#### RQ.SRS-007.LDAP.Authentication +version: 1.0 + +[ClickHouse] SHALL support user authentication via an [LDAP] server. + +#### RQ.SRS-007.LDAP.Authentication.MultipleServers +version: 1.0 + +[ClickHouse] SHALL support specifying multiple [LDAP] servers that can be used to authenticate +users. + +#### RQ.SRS-007.LDAP.Authentication.Protocol.PlainText +version: 1.0 + +[ClickHouse] SHALL support user authentication using plain text `ldap://` non secure protocol. + +#### RQ.SRS-007.LDAP.Authentication.Protocol.TLS +version: 1.0 + +[ClickHouse] SHALL support user authentication using `SSL/TLS` `ldaps://` secure protocol. + +#### RQ.SRS-007.LDAP.Authentication.Protocol.StartTLS +version: 1.0 + +[ClickHouse] SHALL support user authentication using legacy `StartTLS` protocol which is a +plain text `ldap://` protocol that is upgraded to [TLS]. + +#### RQ.SRS-007.LDAP.Authentication.TLS.Certificate.Validation +version: 1.0 + +[ClickHouse] SHALL support certificate validation used for [TLS] connections. + +#### RQ.SRS-007.LDAP.Authentication.TLS.Certificate.SelfSigned +version: 1.0 + +[ClickHouse] SHALL support self-signed certificates for [TLS] connections. + +#### RQ.SRS-007.LDAP.Authentication.TLS.Certificate.SpecificCertificationAuthority +version: 1.0 + +[ClickHouse] SHALL support certificates signed by specific Certification Authority for [TLS] connections. + +#### RQ.SRS-007.LDAP.Server.Configuration.Invalid +version: 1.0 + +[ClickHouse] SHALL return an error and prohibit user login if [LDAP] server configuration is not valid. + +#### RQ.SRS-007.LDAP.User.Configuration.Invalid +version: 1.0 + +[ClickHouse] SHALL return an error and prohibit user login if user configuration is not valid. + +#### RQ.SRS-007.LDAP.Authentication.Mechanism.Anonymous +version: 1.0 + +[ClickHouse] SHALL return an error and prohibit authentication using [Anonymous Authentication Mechanism of Simple Bind] +authentication mechanism. + +#### RQ.SRS-007.LDAP.Authentication.Mechanism.Unauthenticated +version: 1.0 + +[ClickHouse] SHALL return an error and prohibit authentication using [Unauthenticated Authentication Mechanism of Simple Bind] +authentication mechanism. + +#### RQ.SRS-007.LDAP.Authentication.Mechanism.NamePassword +version: 1.0 + +[ClickHouse] SHALL allow authentication using only [Name/Password Authentication Mechanism of Simple Bind] +authentication mechanism. + +#### RQ.SRS-007.LDAP.Authentication.Valid +version: 1.0 + +[ClickHouse] SHALL only allow user authentication using [LDAP] server if and only if +user name and password match [LDAP] server records for the user. + +#### RQ.SRS-007.LDAP.Authentication.Invalid +version: 1.0 + +[ClickHouse] SHALL return an error and prohibit authentication if either user name or password +do not match [LDAP] server records for the user. + +#### RQ.SRS-007.LDAP.Authentication.Invalid.DeletedUser +version: 1.0 + +[ClickHouse] SHALL return an error and prohibit authentication if the user +has been deleted from the [LDAP] server. + +#### RQ.SRS-007.LDAP.Authentication.UsernameChanged +version: 1.0 + +[ClickHouse] SHALL return an error and prohibit authentication if the username is changed +on the [LDAP] server. + +#### RQ.SRS-007.LDAP.Authentication.PasswordChanged +version: 1.0 + +[ClickHouse] SHALL return an error and prohibit authentication if the password +for the user is changed on the [LDAP] server. + +#### RQ.SRS-007.LDAP.Authentication.LDAPServerRestart +version: 1.0 + +[ClickHouse] SHALL support authenticating users after [LDAP] server is restarted. + +#### RQ.SRS-007.LDAP.Authentication.ClickHouseServerRestart +version: 1.0 + +[ClickHouse] SHALL support authenticating users after server is restarted. + +#### RQ.SRS-007.LDAP.Authentication.Parallel +version: 1.0 + +[ClickHouse] SHALL support parallel authentication of users using [LDAP] server. + +#### RQ.SRS-007.LDAP.Authentication.Parallel.ValidAndInvalid +version: 1.0 + +[ClickHouse] SHALL support authentication of valid users and +prohibit authentication of invalid users using [LDAP] server +in parallel without having invalid attempts affecting valid authentications. + +### Specific + +#### RQ.SRS-007.LDAP.UnreachableServer +version: 1.0 + +[ClickHouse] SHALL return an error and prohibit user login if [LDAP] server is unreachable. + +#### RQ.SRS-007.LDAP.Configuration.Server.Name +version: 1.0 + +[ClickHouse] SHALL not support empty string as a server name. + +#### RQ.SRS-007.LDAP.Configuration.Server.Host +version: 1.0 + +[ClickHouse] SHALL support `<host>` parameter to specify [LDAP] +server hostname or IP, this parameter SHALL be mandatory and SHALL not be empty. + +#### RQ.SRS-007.LDAP.Configuration.Server.Port +version: 1.0 + +[ClickHouse] SHALL support `<port>` parameter to specify [LDAP] server port. + +#### RQ.SRS-007.LDAP.Configuration.Server.Port.Default +version: 1.0 + +[ClickHouse] SHALL use default port number `636` if `enable_tls` is set to `yes` or `389` otherwise. + +#### RQ.SRS-007.LDAP.Configuration.Server.AuthDN.Prefix +version: 1.0 + +[ClickHouse] SHALL support `<auth_dn_prefix>` parameter to specify the prefix +of value used to construct the DN to bound to during authentication via [LDAP] server. + +#### RQ.SRS-007.LDAP.Configuration.Server.AuthDN.Suffix +version: 1.0 + +[ClickHouse] SHALL support `<auth_dn_suffix>` parameter to specify the suffix +of value used to construct the DN to bound to during authentication via [LDAP] server. + +#### RQ.SRS-007.LDAP.Configuration.Server.AuthDN.Value +version: 1.0 + +[ClickHouse] SHALL construct DN as `auth_dn_prefix + escape(user_name) + auth_dn_suffix` string. + +> This implies that auth_dn_suffix should usually have comma ',' as its first non-space character. + +#### RQ.SRS-007.LDAP.Configuration.Server.EnableTLS +version: 1.0 + +[ClickHouse] SHALL support `<enable_tls>` parameter to trigger the use of secure connection to the [LDAP] server. + +#### RQ.SRS-007.LDAP.Configuration.Server.EnableTLS.Options.Default +version: 1.0 + +[ClickHouse] SHALL use `yes` value as the default for `<enable_tls>` parameter +to enable SSL/TLS `ldaps://` protocol. + +#### RQ.SRS-007.LDAP.Configuration.Server.EnableTLS.Options.No +version: 1.0 + +[ClickHouse] SHALL support specifying `no` as the value of `<enable_tls>` parameter to enable +plain text `ldap://` protocol. + +#### RQ.SRS-007.LDAP.Configuration.Server.EnableTLS.Options.Yes +version: 1.0 + +[ClickHouse] SHALL support specifying `yes` as the value of `<enable_tls>` parameter to enable +SSL/TLS `ldaps://` protocol. + +#### RQ.SRS-007.LDAP.Configuration.Server.EnableTLS.Options.StartTLS +version: 1.0 + +[ClickHouse] SHALL support specifying `starttls` as the value of `<enable_tls>` parameter to enable +legacy `StartTLS` protocol that used plain text `ldap://` protocol, upgraded to [TLS]. + +#### RQ.SRS-007.LDAP.Configuration.Server.TLSMinimumProtocolVersion +version: 1.0 + +[ClickHouse] SHALL support `<tls_minimum_protocol_version>` parameter to specify +the minimum protocol version of SSL/TLS. + +#### RQ.SRS-007.LDAP.Configuration.Server.TLSMinimumProtocolVersion.Values +version: 1.0 + +[ClickHouse] SHALL support specifying `ssl2`, `ssl3`, `tls1.0`, `tls1.1`, and `tls1.2` +as a value of the `<tls_minimum_protocol_version>` parameter. + +#### RQ.SRS-007.LDAP.Configuration.Server.TLSMinimumProtocolVersion.Default +version: 1.0 + +[ClickHouse] SHALL set `tls1.2` as the default value of the `<tls_minimum_protocol_version>` parameter. + +#### RQ.SRS-007.LDAP.Configuration.Server.TLSRequireCert +version: 1.0 + +[ClickHouse] SHALL support `<tls_require_cert>` parameter to specify [TLS] peer +certificate verification behavior. + +#### RQ.SRS-007.LDAP.Configuration.Server.TLSRequireCert.Options.Default +version: 1.0 + +[ClickHouse] SHALL use `demand` value as the default for the `<tls_require_cert>` parameter. + +#### RQ.SRS-007.LDAP.Configuration.Server.TLSRequireCert.Options.Demand +version: 1.0 + +[ClickHouse] SHALL support specifying `demand` as the value of `<tls_require_cert>` parameter to +enable requesting of client certificate. If no certificate is provided, or a bad certificate is +provided, the session SHALL be immediately terminated. + +#### RQ.SRS-007.LDAP.Configuration.Server.TLSRequireCert.Options.Allow +version: 1.0 + +[ClickHouse] SHALL support specifying `allow` as the value of `<tls_require_cert>` parameter to +enable requesting of client certificate. If no +certificate is provided, the session SHALL proceed normally. +If a bad certificate is provided, it SHALL be ignored and the session SHALL proceed normally. + +#### RQ.SRS-007.LDAP.Configuration.Server.TLSRequireCert.Options.Try +version: 1.0 + +[ClickHouse] SHALL support specifying `try` as the value of `<tls_require_cert>` parameter to +enable requesting of client certificate. If no certificate is provided, the session +SHALL proceed normally. If a bad certificate is provided, the session SHALL be +immediately terminated. + +#### RQ.SRS-007.LDAP.Configuration.Server.TLSRequireCert.Options.Never +version: 1.0 + +[ClickHouse] SHALL support specifying `never` as the value of `<tls_require_cert>` parameter to +disable requesting of client certificate. + +#### RQ.SRS-007.LDAP.Configuration.Server.TLSCertFile +version: 1.0 + +[ClickHouse] SHALL support `<tls_cert_file>` to specify the path to certificate file used by +[ClickHouse] to establish connection with the [LDAP] server. + +#### RQ.SRS-007.LDAP.Configuration.Server.TLSKeyFile +version: 1.0 + +[ClickHouse] SHALL support `<tls_key_file>` to specify the path to key file for the certificate +specified by the `<tls_cert_file>` parameter. + +#### RQ.SRS-007.LDAP.Configuration.Server.TLSCACertDir +version: 1.0 + +[ClickHouse] SHALL support `<tls_ca_cert_dir>` parameter to specify to a path to +the directory containing [CA] certificates used to verify certificates provided by the [LDAP] server. + +#### RQ.SRS-007.LDAP.Configuration.Server.TLSCACertFile +version: 1.0 + +[ClickHouse] SHALL support `<tls_ca_cert_file>` parameter to specify a path to a specific +[CA] certificate file used to verify certificates provided by the [LDAP] server. + +#### RQ.SRS-007.LDAP.Configuration.Server.TLSCipherSuite +version: 1.0 + +[ClickHouse] SHALL support `tls_cipher_suite` parameter to specify allowed cipher suites. +The value SHALL use the same format as the `ciphersuites` in the [OpenSSL Ciphers]. + +For example, + +```xml +<tls_cipher_suite>ECDHE-ECDSA-AES256-GCM-SHA384:ECDHE-RSA-AES256-GCM-SHA384:AES256-GCM-SHA384</tls_cipher_suite> +``` + +The available suites SHALL depend on the [OpenSSL] library version and variant used to build +[ClickHouse] and therefore might change. + +#### RQ.SRS-007.LDAP.Configuration.Server.Syntax +version: 1.0 + +[ClickHouse] SHALL support the following example syntax to create an entry for an [LDAP] server inside the `config.xml` +configuration file or of any configuration file inside the `config.d` directory. + +```xml +<yandex> + <my_ldap_server> + <host>localhost</host> + <port>636</port> + <auth_dn_prefix>cn=</auth_dn_prefix> + <auth_dn_suffix>, ou=users, dc=example, dc=com</auth_dn_suffix> + <enable_tls>yes</enable_tls> + <tls_minimum_protocol_version>tls1.2</tls_minimum_protocol_version> + <tls_require_cert>demand</tls_require_cert> + <tls_cert_file>/path/to/tls_cert_file</tls_cert_file> + <tls_key_file>/path/to/tls_key_file</tls_key_file> + <tls_ca_cert_file>/path/to/tls_ca_cert_file</tls_ca_cert_file> + <tls_ca_cert_dir>/path/to/tls_ca_cert_dir</tls_ca_cert_dir> + <tls_cipher_suite>ECDHE-ECDSA-AES256-GCM-SHA384:ECDHE-RSA-AES256-GCM-SHA384:AES256-GCM-SHA384</tls_cipher_suite> + </my_ldap_server> +</yandex> +``` + +#### RQ.SRS-007.LDAP.Configuration.User.RBAC +version: 1.0 + +[ClickHouse] SHALL support creating users identified using an [LDAP] server using +the following RBAC command + +```sql +CREATE USER name IDENTIFIED WITH ldap_server BY 'server_name' +``` + +#### RQ.SRS-007.LDAP.Configuration.User.Syntax +version: 1.0 + +[ClickHouse] SHALL support the following example syntax to create a user that is authenticated using +an [LDAP] server inside the `users.xml` file or any configuration file inside the `users.d` directory. + +```xml +<yandex> + <users> + <user_name> + <ldap> + <server>my_ldap_server</server> + </ldap> + </user_name> + </users> +</yandex> +``` + +#### RQ.SRS-007.LDAP.Configuration.User.Name.Empty +version: 1.0 + +[ClickHouse] SHALL not support empty string as a user name. + +#### RQ.SRS-007.LDAP.Configuration.User.BothPasswordAndLDAP +version: 1.0 + +[ClickHouse] SHALL throw an error if `<ldap>` is specified for the user and at the same +time user configuration contains any of the `<password*>` entries. + +#### RQ.SRS-007.LDAP.Configuration.User.LDAP.InvalidServerName.NotDefined +version: 1.0 + +[ClickHouse] SHALL throw an error during any authentification attempt +if the name of the [LDAP] server used inside the `<ldap>` entry +is not defined in the `<ldap_servers>` section. + +#### RQ.SRS-007.LDAP.Configuration.User.LDAP.InvalidServerName.Empty +version: 1.0 + +[ClickHouse] SHALL throw an error during any authentification attempt +if the name of the [LDAP] server used inside the `<ldap>` entry +is empty. + +#### RQ.SRS-007.LDAP.Configuration.User.OnlyOneServer +version: 1.0 + +[ClickHouse] SHALL support specifying only one [LDAP] server for a given user. + +#### RQ.SRS-007.LDAP.Configuration.User.Name.Long +version: 1.0 + +[ClickHouse] SHALL support long user names of at least 256 bytes +to specify users that can be authenticated using an [LDAP] server. + +#### RQ.SRS-007.LDAP.Configuration.User.Name.UTF8 +version: 1.0 + +[ClickHouse] SHALL support user names that contain [UTF-8] characters. + +#### RQ.SRS-007.LDAP.Authentication.Username.Empty +version: 1.0 + +[ClickHouse] SHALL not support authenticating users with empty username. + +#### RQ.SRS-007.LDAP.Authentication.Username.Long +version: 1.0 + +[ClickHouse] SHALL support authenticating users with a long username of at least 256 bytes. + +#### RQ.SRS-007.LDAP.Authentication.Username.UTF8 +version: 1.0 + +[ClickHouse] SHALL support authentication users with a username that contains [UTF-8] characters. + +#### RQ.SRS-007.LDAP.Authentication.Password.Empty +version: 1.0 + +[ClickHouse] SHALL not support authenticating users with empty passwords +even if an empty password is valid for the user and +is allowed by the [LDAP] server. + +#### RQ.SRS-007.LDAP.Authentication.Password.Long +version: 1.0 + +[ClickHouse] SHALL support long password of at least 256 bytes +that can be used to authenticate users using an [LDAP] server. + +#### RQ.SRS-007.LDAP.Authentication.Password.UTF8 +version: 1.0 + +[ClickHouse] SHALL support [UTF-8] characters in passwords +used to authenticate users using an [LDAP] server. + +## References + +* **ClickHouse:** https://clickhouse.tech +* **GitHub repository:** https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/ldap/requirements/requirements.md +* **Revision history:** https://github.com/ClickHouse/ClickHouse/commits/master/tests/testflows/ldap/requirements/requirements.md +* **Git:** https://git-scm.com/ + +[Anonymous Authentication Mechanism of Simple Bind]: https://ldapwiki.com/wiki/Simple%20Authentication#section-Simple+Authentication-AnonymousAuthenticationMechanismOfSimpleBind +[Unauthenticated Authentication Mechanism of Simple Bind]: https://ldapwiki.com/wiki/Simple%20Authentication#section-Simple+Authentication-UnauthenticatedAuthenticationMechanismOfSimpleBind +[Name/Password Authentication Mechanism of Simple Bind]: https://ldapwiki.com/wiki/Simple%20Authentication#section-Simple+Authentication-NamePasswordAuthenticationMechanismOfSimpleBind +[UTF-8]: https://en.wikipedia.org/wiki/UTF-8 +[OpenSSL]: https://www.openssl.org/ +[OpenSSL Ciphers]: https://www.openssl.org/docs/manmaster/man1/openssl-ciphers.html +[CA]: https://en.wikipedia.org/wiki/Certificate_authority +[TLS]: https://en.wikipedia.org/wiki/Transport_Layer_Security +[LDAP]: https://en.wikipedia.org/wiki/Lightweight_Directory_Access_Protocol +[ClickHouse]: https://clickhouse.tech +[GitHub]: https://github.com +[GitHub Repository]: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/ldap/requirements/requirements.md +[Revision History]: https://github.com/ClickHouse/ClickHouse/commits/master/tests/testflows/ldap/requirements/requirements.md +[Git]: https://git-scm.com/ diff --git a/tests/testflows/ldap/requirements/requirements.py b/tests/testflows/ldap/requirements/requirements.py index 92491f4318b..967e097d1fa 100644 --- a/tests/testflows/ldap/requirements/requirements.py +++ b/tests/testflows/ldap/requirements/requirements.py @@ -1,6 +1,6 @@ # These requirements were auto generated # from software requirements specification (SRS) -# document by TestFlows v1.6.200623.1103543. +# document by TestFlows v1.6.200811.1124123. # Do not edit by hand but re-generate instead # using 'tfs requirements generate' command. from testflows.core import Requirement @@ -243,7 +243,7 @@ RQ_SRS_007_LDAP_Authentication_PasswordChanged = Requirement( type=None, uid=None, description=( - '[ClickHouse] SHALL return an error and prohibit authentication if the password \n' + '[ClickHouse] SHALL return an error and prohibit authentication if the password\n' 'for the user is changed on the [LDAP] server.\n' ), link=None @@ -296,8 +296,8 @@ RQ_SRS_007_LDAP_Authentication_Parallel_ValidAndInvalid = Requirement( type=None, uid=None, description=( - '[ClickHouse] SHALL support authentication of valid users and \n' - 'prohibit authentication of invalid users using [LDAP] server \n' + '[ClickHouse] SHALL support authentication of valid users and\n' + 'prohibit authentication of invalid users using [LDAP] server\n' 'in parallel without having invalid attempts affecting valid authentications.\n' ), link=None @@ -338,7 +338,7 @@ RQ_SRS_007_LDAP_Configuration_Server_Host = Requirement( uid=None, description=( '[ClickHouse] SHALL support `<host>` parameter to specify [LDAP]\n' - 'server hostname or IP, this parameter SHALL be mandatory and SHALL not be empty. \n' + 'server hostname or IP, this parameter SHALL be mandatory and SHALL not be empty.\n' ), link=None ) @@ -391,7 +391,7 @@ RQ_SRS_007_LDAP_Configuration_Server_AuthDN_Suffix = Requirement( type=None, uid=None, description=( - '[ClickHouse] SHALL support `<auth_dn_suffix>` parameter to specify the suffix \n' + '[ClickHouse] SHALL support `<auth_dn_suffix>` parameter to specify the suffix\n' 'of value used to construct the DN to bound to during authentication via [LDAP] server.\n' ), link=None @@ -420,7 +420,7 @@ RQ_SRS_007_LDAP_Configuration_Server_EnableTLS = Requirement( type=None, uid=None, description=( - '[ClickHouse] SHALL support `<enable_tls>` parameter to trigger the use of secure connection to the [LDAP] server. \n' + '[ClickHouse] SHALL support `<enable_tls>` parameter to trigger the use of secure connection to the [LDAP] server.\n' ), link=None ) @@ -434,7 +434,7 @@ RQ_SRS_007_LDAP_Configuration_Server_EnableTLS_Options_Default = Requirement( uid=None, description=( '[ClickHouse] SHALL use `yes` value as the default for `<enable_tls>` parameter\n' - 'to enable SSL/TLS `ldaps://` protocol. \n' + 'to enable SSL/TLS `ldaps://` protocol.\n' ), link=None ) @@ -447,7 +447,7 @@ RQ_SRS_007_LDAP_Configuration_Server_EnableTLS_Options_No = Requirement( type=None, uid=None, description=( - '[ClickHouse] SHALL support specifying `no` as the value of `<enable_tls>` parameter to enable \n' + '[ClickHouse] SHALL support specifying `no` as the value of `<enable_tls>` parameter to enable\n' 'plain text `ldap://` protocol.\n' ), link=None @@ -461,7 +461,7 @@ RQ_SRS_007_LDAP_Configuration_Server_EnableTLS_Options_Yes = Requirement( type=None, uid=None, description=( - '[ClickHouse] SHALL support specifying `yes` as the value of `<enable_tls>` parameter to enable \n' + '[ClickHouse] SHALL support specifying `yes` as the value of `<enable_tls>` parameter to enable\n' 'SSL/TLS `ldaps://` protocol.\n' ), link=None @@ -475,7 +475,7 @@ RQ_SRS_007_LDAP_Configuration_Server_EnableTLS_Options_StartTLS = Requirement( type=None, uid=None, description=( - '[ClickHouse] SHALL support specifying `starttls` as the value of `<enable_tls>` parameter to enable \n' + '[ClickHouse] SHALL support specifying `starttls` as the value of `<enable_tls>` parameter to enable\n' 'legacy `StartTLS` protocol that used plain text `ldap://` protocol, upgraded to [TLS].\n' ), link=None @@ -489,7 +489,7 @@ RQ_SRS_007_LDAP_Configuration_Server_TLSMinimumProtocolVersion = Requirement( type=None, uid=None, description=( - '[ClickHouse] SHALL support `<tls_minimum_protocol_version>` parameter to specify \n' + '[ClickHouse] SHALL support `<tls_minimum_protocol_version>` parameter to specify\n' 'the minimum protocol version of SSL/TLS.\n' ), link=None @@ -517,7 +517,7 @@ RQ_SRS_007_LDAP_Configuration_Server_TLSMinimumProtocolVersion_Default = Require type=None, uid=None, description=( - '[ClickHouse] SHALL set `tls1.2` as the default value of the `<tls_minimum_protocol_version>` parameter. \n' + '[ClickHouse] SHALL set `tls1.2` as the default value of the `<tls_minimum_protocol_version>` parameter.\n' ), link=None ) @@ -530,7 +530,7 @@ RQ_SRS_007_LDAP_Configuration_Server_TLSRequireCert = Requirement( type=None, uid=None, description=( - '[ClickHouse] SHALL support `<tls_require_cert>` parameter to specify [TLS] peer \n' + '[ClickHouse] SHALL support `<tls_require_cert>` parameter to specify [TLS] peer\n' 'certificate verification behavior.\n' ), link=None @@ -574,7 +574,7 @@ RQ_SRS_007_LDAP_Configuration_Server_TLSRequireCert_Options_Allow = Requirement( description=( '[ClickHouse] SHALL support specifying `allow` as the value of `<tls_require_cert>` parameter to\n' 'enable requesting of client certificate. If no\n' - 'certificate is provided, the session SHALL proceed normally. \n' + 'certificate is provided, the session SHALL proceed normally.\n' 'If a bad certificate is provided, it SHALL be ignored and the session SHALL proceed normally.\n' ), link=None @@ -590,7 +590,7 @@ RQ_SRS_007_LDAP_Configuration_Server_TLSRequireCert_Options_Try = Requirement( description=( '[ClickHouse] SHALL support specifying `try` as the value of `<tls_require_cert>` parameter to\n' 'enable requesting of client certificate. If no certificate is provided, the session\n' - 'SHALL proceed normally. If a bad certificate is provided, the session SHALL be \n' + 'SHALL proceed normally. If a bad certificate is provided, the session SHALL be\n' 'immediately terminated.\n' ), link=None @@ -646,7 +646,7 @@ RQ_SRS_007_LDAP_Configuration_Server_TLSCACertDir = Requirement( type=None, uid=None, description=( - '[ClickHouse] SHALL support `<tls_ca_cert_dir>` parameter to specify to a path to \n' + '[ClickHouse] SHALL support `<tls_ca_cert_dir>` parameter to specify to a path to\n' 'the directory containing [CA] certificates used to verify certificates provided by the [LDAP] server.\n' ), link=None @@ -660,7 +660,7 @@ RQ_SRS_007_LDAP_Configuration_Server_TLSCACertFile = Requirement( type=None, uid=None, description=( - '[ClickHouse] SHALL support `<tls_ca_cert_file>` parameter to specify a path to a specific \n' + '[ClickHouse] SHALL support `<tls_ca_cert_file>` parameter to specify a path to a specific\n' '[CA] certificate file used to verify certificates provided by the [LDAP] server.\n' ), link=None @@ -677,7 +677,7 @@ RQ_SRS_007_LDAP_Configuration_Server_TLSCipherSuite = Requirement( '[ClickHouse] SHALL support `tls_cipher_suite` parameter to specify allowed cipher suites.\n' 'The value SHALL use the same format as the `ciphersuites` in the [OpenSSL Ciphers].\n' '\n' - 'For example, \n' + 'For example,\n' '\n' '```xml\n' '<tls_cipher_suite>ECDHE-ECDSA-AES256-GCM-SHA384:ECDHE-RSA-AES256-GCM-SHA384:AES256-GCM-SHA384</tls_cipher_suite>\n' @@ -722,6 +722,24 @@ RQ_SRS_007_LDAP_Configuration_Server_Syntax = Requirement( link=None ) +RQ_SRS_007_LDAP_Configuration_User_RBAC = Requirement( + name='RQ.SRS-007.LDAP.Configuration.User.RBAC', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support creating users identified using an [LDAP] server using\n' + 'the following RBAC command\n' + '\n' + '```sql\n' + "CREATE USER name IDENTIFIED WITH ldap_server BY 'server_name'\n" + '```\n' + ), + link=None + ) + RQ_SRS_007_LDAP_Configuration_User_Syntax = Requirement( name='RQ.SRS-007.LDAP.Configuration.User.Syntax', version='1.0', @@ -769,7 +787,7 @@ RQ_SRS_007_LDAP_Configuration_User_BothPasswordAndLDAP = Requirement( type=None, uid=None, description=( - '[ClickHouse] SHALL throw an error if `<ldap>` is specified for the user and at the same \n' + '[ClickHouse] SHALL throw an error if `<ldap>` is specified for the user and at the same\n' 'time user configuration contains any of the `<password*>` entries.\n' ), link=None @@ -784,8 +802,8 @@ RQ_SRS_007_LDAP_Configuration_User_LDAP_InvalidServerName_NotDefined = Requireme uid=None, description=( '[ClickHouse] SHALL throw an error during any authentification attempt\n' - 'if the name of the [LDAP] server used inside the `<ldap>` entry \n' - 'is not defined in the `<ldap_servers>` section. \n' + 'if the name of the [LDAP] server used inside the `<ldap>` entry\n' + 'is not defined in the `<ldap_servers>` section.\n' ), link=None ) @@ -800,7 +818,7 @@ RQ_SRS_007_LDAP_Configuration_User_LDAP_InvalidServerName_Empty = Requirement( description=( '[ClickHouse] SHALL throw an error during any authentification attempt\n' 'if the name of the [LDAP] server used inside the `<ldap>` entry\n' - 'is empty. \n' + 'is empty.\n' ), link=None ) diff --git a/tests/testflows/ldap/tests/authentications.py b/tests/testflows/ldap/tests/authentications.py index 949f8444c21..c56c23d6813 100644 --- a/tests/testflows/ldap/tests/authentications.py +++ b/tests/testflows/ldap/tests/authentications.py @@ -458,7 +458,7 @@ def empty_username_and_empty_password(self, server=None, rbac=False): ) @Examples("rbac", [ (False,), - (True,) + (True, Requirements(RQ_SRS_007_LDAP_Configuration_User_RBAC("1.0"))) ]) def feature(self, rbac, servers=None, node="clickhouse1"): """Check that users can be authenticated using an LDAP server when From 83a3f589437208fb37ed1ee5b16915459f729056 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu, 13 Aug 2020 01:46:40 +0300 Subject: [PATCH 299/374] Add results from AWS m5a.8xlarge --- .../hardware/results/051_aws_m5a_8xlarge.json | 55 +++++++++++++++++++ 1 file changed, 55 insertions(+) create mode 100644 website/benchmark/hardware/results/051_aws_m5a_8xlarge.json diff --git a/website/benchmark/hardware/results/051_aws_m5a_8xlarge.json b/website/benchmark/hardware/results/051_aws_m5a_8xlarge.json new file mode 100644 index 00000000000..8a231be0ede --- /dev/null +++ b/website/benchmark/hardware/results/051_aws_m5a_8xlarge.json @@ -0,0 +1,55 @@ +[ + { + "system": "AWS m5a.8xlarge", + "system_full": "AWS m5a.8xlarge 32 vCPU, 128 GiB RAM, EBS", + "time": "2020-08-13 00:00:00", + "kind": "cloud", + "result": + [ + [0.015, 0.001, 0.001], + [0.038, 0.015, 0.015], + [0.072, 0.033, 0.033], + [0.198, 0.052, 0.051], + [0.338, 0.166, 0.120], + [1.036, 0.252, 0.264], + [0.034, 0.020, 0.020], + [0.020, 0.015, 0.015], + [0.521, 0.414, 0.391], + [0.563, 0.496, 0.497], + [0.242, 0.143, 0.143], + [0.261, 0.168, 0.168], + [1.067, 0.574, 0.567], + [1.537, 0.772, 0.759], + [0.802, 0.736, 0.752], + [0.904, 0.797, 0.742], + [2.229, 1.795, 1.867], + [1.314, 0.987, 0.962], + [5.216, 4.149, 5.540], + [0.208, 0.057, 0.036], + [9.238, 0.660, 0.609], + [10.105, 0.685, 0.668], + [19.544, 1.851, 1.766], + [19.690, 0.943, 0.817], + [1.834, 0.228, 0.217], + [0.757, 0.177, 0.176], + [2.331, 0.224, 0.214], + [9.174, 0.728, 0.688], + [7.330, 1.130, 1.108], + [1.834, 1.810, 1.818], + [1.429, 0.547, 0.550], + [4.518, 0.879, 0.895], + [6.157, 5.540, 5.547], + [9.846, 3.033, 3.044], + [9.847, 3.061, 3.016], + [1.157, 1.086, 1.117], + [0.238, 0.169, 0.175], + [0.094, 0.072, 0.072], + [0.041, 0.037, 0.041], + [0.453, 0.364, 0.345], + [0.054, 0.015, 0.019], + [0.024, 0.010, 0.010], + [0.012, 0.006, 0.007] + ] + } +] + From 2bed8bed80126dac20b82c9b1cc1728fbdb22bf3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu, 13 Aug 2020 03:34:48 +0300 Subject: [PATCH 300/374] Added a log of running benchmark of MonetDB on AWS m5a.8xlarge #13632 --- benchmark/monetdb/aws.log | 1774 +++++++++++++++++++++++++++++++++++++ 1 file changed, 1774 insertions(+) create mode 100644 benchmark/monetdb/aws.log diff --git a/benchmark/monetdb/aws.log b/benchmark/monetdb/aws.log new file mode 100644 index 00000000000..e5fa40018bf --- /dev/null +++ b/benchmark/monetdb/aws.log @@ -0,0 +1,1774 @@ +milovidov@milovidov-desktop:~$ ssh -i ▉▉▉.pem ubuntu@ec2-▉▉▉.us-east-2.compute.amazonaws.com +Welcome to Ubuntu 18.04.5 LTS (GNU/Linux 5.3.0-1032-aws x86_64) + + * Documentation: https://help.ubuntu.com + * Management: https://landscape.canonical.com + * Support: https://ubuntu.com/advantage + + System information as of Wed Aug 12 22:11:00 UTC 2020 + + System load: 0.06 Processes: 318 + Usage of /: 2.3% of 193.82GB Users logged in: 1 + Memory usage: 0% IP address for ens5: ▉▉▉ + Swap usage: 0% + + +0 packages can be updated. +0 updates are security updates. + + +Last login: Wed Aug 12 22:00:27 2020 from 109.252.51.97 +To run a command as administrator (user "root"), use "sudo <command>". +See "man sudo_root" for details. + +ubuntu@ip-▉▉▉:~$ wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/benchmark-new.sh +--2020-08-12 22:11:07-- https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/benchmark-new.sh +Resolving raw.githubusercontent.com (raw.githubusercontent.com)... 151.101.248.133 +Connecting to raw.githubusercontent.com (raw.githubusercontent.com)|151.101.248.133|:443... connected. +HTTP request sent, awaiting response... 200 OK +Length: 468 [text/plain] +Saving to: ‘benchmark-new.sh’ + +benchmark-new.sh 100%[========================================================================================================================>] 468 --.-KB/s in 0s + +2020-08-12 22:11:12 (44.9 MB/s) - ‘benchmark-new.sh’ saved [468/468] + +ubuntu@ip-▉▉▉:~$ chmod a+x benchmark-new.sh +ubuntu@ip-▉▉▉:~$ wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/queries.sql +--2020-08-12 22:11:12-- https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/queries.sql +Resolving raw.githubusercontent.com (raw.githubusercontent.com)... 151.101.248.133 +Connecting to raw.githubusercontent.com (raw.githubusercontent.com)|151.101.248.133|:443... connected. +HTTP request sent, awaiting response... 200 OK +Length: 8074 (7.9K) [text/plain] +Saving to: ‘queries.sql’ + +queries.sql 100%[========================================================================================================================>] 7.88K --.-KB/s in 0s + +2020-08-12 22:11:12 (135 MB/s) - ‘queries.sql’ saved [8074/8074] + +ubuntu@ip-▉▉▉:~$ wget https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_100m_obfuscated_v1.tar.xz +--2020-08-12 22:11:27-- https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_100m_obfuscated_v1.tar.xz +Resolving clickhouse-datasets.s3.yandex.net (clickhouse-datasets.s3.yandex.net)... 93.158.134.158, 2a02:6b8::2:158 +Connecting to clickhouse-datasets.s3.yandex.net (clickhouse-datasets.s3.yandex.net)|93.158.134.158|:443... connected. +HTTP request sent, awaiting response... 200 OK +Length: 9722280160 (9.1G) [application/octet-stream] +Saving to: ‘hits_100m_obfuscated_v1.tar.xz’ + +hits_100m_obfuscated_v1.tar.xz 100%[========================================================================================================================>] 9.05G 10.8MB/s in 13m 7s + +2020-08-12 22:24:35 (11.8 MB/s) - ‘hits_100m_obfuscated_v1.tar.xz’ saved [9722280160/9722280160] + +ubuntu@ip-▉▉▉:~$ tar xvf hits_100m_obfuscated_v1.tar.xz -C . +hits_100m_obfuscated_v1/data/ +hits_100m_obfuscated_v1/data/default/ +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/ +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsParameter.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/WithHash.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/WatchID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Interests.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SocialNetwork.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsLink.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/RefererCategoryID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ResponseStartTiming.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ParamCurrency.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/FetchTiming.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/MobilePhone.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/HistoryLength.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Params.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/CookieEnable.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ParamOrderID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Refresh.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/HasGCLID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsNotBounce.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/EventDate.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SocialAction.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/RemoteIP.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ResolutionWidth.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsEvent.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/WithHash.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SocialSourceNetworkID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/CLID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/DontCountHits.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/EventTime.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/FlashMinor.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UTMMedium.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/WatchID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/FlashMinor2.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/CLID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/URLCategoryID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/URLRegionID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsEvent.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsParameter.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/WindowClientHeight.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SocialSourcePage.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/checksums.txt +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/CounterID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SilverlightVersion1.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SocialSourceNetworkID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Sex.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/PageCharset.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UserAgentMajor.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ResolutionDepth.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/OpenstatCampaignID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IPNetworkID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/OriginalURL.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsArtifical.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/CounterClass.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Refresh.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UTMSource.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/BrowserLanguage.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/HTTPError.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/NetMajor.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UTMTerm.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/OpenerName.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Robotness.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/GoodEvent.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/FetchTiming.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UserAgent.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SilverlightVersion2.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ClientEventTime.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/EventTime.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/WindowName.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/OriginalURL.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Params.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SearchEngineID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/URLRegionID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Age.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ParamCurrencyID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/FlashMajor.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/FUniqID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SilverlightVersion3.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UTMTerm.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/EventDate.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SilverlightVersion4.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/URLCategoryID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsOldCounter.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/WindowClientHeight.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Referer.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/RefererCategoryID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsDownload.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ClientTimeZone.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ParamPrice.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UTMCampaign.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsArtifical.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/HitColor.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/HistoryLength.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/RegionID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsLink.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Title.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/FUniqID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SilverlightVersion2.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ConnectTiming.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ResolutionWidth.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/LocalEventTime.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/columns.txt +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/minmax_EventDate.idx +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UTMMedium.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/TraficSourceID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ClientTimeZone.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/OpenstatSourceID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/NetMinor.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/RegionID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/HTTPError.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/URLHash.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsMobile.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/CodeVersion.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Age.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/URL.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/JavascriptEnable.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsOldCounter.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/FlashMajor.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/WindowName.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/JavaEnable.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/CounterClass.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SocialSourcePage.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/RefererRegionID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/partition.dat +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/DNSTiming.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/OpenstatServiceName.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UserAgentMinor.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/MobilePhoneModel.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/OpenstatServiceName.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ResolutionHeight.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ParamCurrency.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Robotness.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SendTiming.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ConnectTiming.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/FromTag.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/OpenstatAdID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/URLHash.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/DontCountHits.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SendTiming.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ResponseEndTiming.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ParamOrderID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UTMCampaign.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsMobile.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/MobilePhone.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/RefererHash.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Referer.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/BrowserCountry.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/CookieEnable.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UserAgentMinor.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/FlashMinor.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/GoodEvent.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/AdvEngineID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/OS.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/HasGCLID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/OS.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ResolutionHeight.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ResolutionDepth.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/BrowserLanguage.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/MobilePhoneModel.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/OpenstatSourceID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsDownload.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Sex.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ClientIP.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/DNSTiming.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/FlashMinor2.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/count.txt +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ClientEventTime.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/LocalEventTime.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UserAgentMajor.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/AdvEngineID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ResponseEndTiming.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/HitColor.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ParamCurrencyID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Title.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/WindowClientWidth.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SilverlightVersion4.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Income.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UTMContent.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/HID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SilverlightVersion1.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/NetMajor.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UserID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/FromTag.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/TraficSourceID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/primary.idx +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UTMContent.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/WindowClientWidth.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UserAgent.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SocialAction.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/JavascriptEnable.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SilverlightVersion3.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/OpenstatCampaignID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/RefererHash.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/OpenerName.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/BrowserCountry.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/JavaEnable.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Interests.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ResponseStartTiming.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/RefererRegionID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/RemoteIP.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IsNotBounce.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ClientIP.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/CounterID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UserID.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SearchPhrase.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/CodeVersion.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SearchPhrase.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/Income.bin +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/ParamPrice.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/PageCharset.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SearchEngineID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/SocialNetwork.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/OpenstatAdID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/URL.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/IPNetworkID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/HID.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/UTMSource.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/201307_1_96_4/NetMinor.mrk2 +hits_100m_obfuscated_v1/data/default/hits_100m_obfuscated/format_version.txt +hits_100m_obfuscated_v1/metadata/ +hits_100m_obfuscated_v1/metadata/default/ +hits_100m_obfuscated_v1/metadata/default/hits_100m_obfuscated.sql +ubuntu@ip-▉▉▉:~$ mv hits_100m_obfuscated_v1/* . +mv: cannot move 'hits_100m_obfuscated_v1/data' to './data': Directory not empty +mv: cannot move 'hits_100m_obfuscated_v1/metadata' to './metadata': Directory not empty +ubuntu@ip-▉▉▉:~$ rm -rf data metadata +ubuntu@ip-▉▉▉:~$ mv hits_100m_obfuscated_v1/* . +ubuntu@ip-▉▉▉:~$ ./clickhouse client --query "SELECT count() FROM hits_100m_obfuscated" +100000000 +ubuntu@ip-▉▉▉:~$ mcedit benchmark-new.sh + +Command 'mcedit' not found, but can be installed with: + +sudo apt install mc + +ubuntu@ip-▉▉▉:~$ nanobenchmark-new.sh +nanobenchmark-new.sh: command not found +ubuntu@ip-▉▉▉:~$ nano benchmark-new.sh +ubuntu@ip-▉▉▉:~$ ./benchmark-new.sh hits_100m_obfuscated +[0.015, 0.001, 0.001], +[0.038, 0.015, 0.015], +[0.072, 0.033, 0.033], +[0.198, 0.052, 0.051], +[0.338, 0.166, 0.120], +[1.036, 0.252, 0.264], +[0.034, 0.020, 0.020], +[0.020, 0.015, 0.015], +[0.521, 0.414, 0.391], +[0.563, 0.496, 0.497], +[0.242, 0.143, 0.143], +[0.261, 0.168, 0.168], +[1.067, 0.574, 0.567], +[1.537, 0.772, 0.759], +[0.802, 0.736, 0.752], +[0.904, 0.797, 0.742], +[2.229, 1.795, 1.867], +[1.314, 0.987, 0.962], +[5.216, 4.149, 5.540], +[0.208, 0.057, 0.036], +[9.238, 0.660, 0.609], +[10.105, 0.685, 0.668], +[19.544, 1.851, 1.766], +[19.690, 0.943, 0.817], +[1.834, 0.228, 0.217], +[0.757, 0.177, 0.176], +[2.331, 0.224, 0.214], +[9.174, 0.728, 0.688], +[7.330, 1.130, 1.108], +[1.834, 1.810, 1.818], +[1.429, 0.547, 0.550], +[4.518, 0.879, 0.895], +[6.157, 5.540, 5.547], +[9.846, 3.033, 3.044], +[9.847, 3.061, 3.016], +[1.157, 1.086, 1.117], +[0.238, 0.169, 0.175], +[0.094, 0.072, 0.072], +[0.041, 0.037, 0.041], +[0.453, 0.364, 0.345], +[0.054, 0.015, 0.019], +[0.024, 0.010, 0.010], +[0.012, 0.006, 0.007], +ubuntu@ip-▉▉▉:~$ ./clickhouse client +ClickHouse client version 20.8.1.4338 (official build). +Connecting to localhost:9000 as user default. +Connected to ClickHouse server version 20.8.1 revision 54438. + +ip-▉▉▉.us-east-2.compute.internal :) SELECT +:-] toInt64(WatchID) = -9223372036854775808 ? -9223372036854775807 : toInt64(WatchID), +:-] toInt8(JavaEnable) = -128 ? -127 : toInt8(JavaEnable), +:-] toValidUTF8(toString(Title)), +:-] toInt16(GoodEvent) = -32768 ? -32767 : toInt16(GoodEvent), +:-] EventTime, +:-] EventDate, +:-] toInt32(CounterID) = -2147483648 ? -2147483647 : toInt32(CounterID), +:-] toInt32(ClientIP) = -2147483648 ? -2147483647 : toInt32(ClientIP), +:-] toInt32(RegionID) = -2147483648 ? -2147483647 : toInt32(RegionID), +:-] toInt64(UserID) = -9223372036854775808 ? -9223372036854775807 : toInt64(UserID), +:-] toInt8(CounterClass) = -128 ? -127 : toInt8(CounterClass), +:-] toInt8(OS) = -128 ? -127 : toInt8(OS), +:-] toInt8(UserAgent) = -128 ? -127 : toInt8(UserAgent), +:-] toValidUTF8(toString(URL)), +:-] toValidUTF8(toString(Referer)), +:-] toInt8(Refresh) = -128 ? -127 : toInt8(Refresh), +:-] toInt16(RefererCategoryID) = -32768 ? -32767 : toInt16(RefererCategoryID), +:-] toInt32(RefererRegionID) = -2147483648 ? -2147483647 : toInt32(RefererRegionID), +:-] toInt16(URLCategoryID) = -32768 ? -32767 : toInt16(URLCategoryID), +:-] toInt32(URLRegionID) = -2147483648 ? -2147483647 : toInt32(URLRegionID), +:-] toInt16(ResolutionWidth) = -32768 ? -32767 : toInt16(ResolutionWidth), +:-] toInt16(ResolutionHeight) = -32768 ? -32767 : toInt16(ResolutionHeight), +:-] toInt8(ResolutionDepth) = -128 ? -127 : toInt8(ResolutionDepth), +:-] toInt8(FlashMajor) = -128 ? -127 : toInt8(FlashMajor), +:-] toInt8(FlashMinor) = -128 ? -127 : toInt8(FlashMinor), +:-] toValidUTF8(toString(FlashMinor2)), +:-] toInt8(NetMajor) = -128 ? -127 : toInt8(NetMajor), +:-] toInt8(NetMinor) = -128 ? -127 : toInt8(NetMinor), +:-] toInt16(UserAgentMajor) = -32768 ? -32767 : toInt16(UserAgentMajor), +:-] toValidUTF8(toString(UserAgentMinor)), +:-] toInt8(CookieEnable) = -128 ? -127 : toInt8(CookieEnable), +:-] toInt8(JavascriptEnable) = -128 ? -127 : toInt8(JavascriptEnable), +:-] toInt8(IsMobile) = -128 ? -127 : toInt8(IsMobile), +:-] toInt8(MobilePhone) = -128 ? -127 : toInt8(MobilePhone), +:-] toValidUTF8(toString(MobilePhoneModel)), +:-] toValidUTF8(toString(Params)), +:-] toInt32(IPNetworkID) = -2147483648 ? -2147483647 : toInt32(IPNetworkID), +:-] toInt8(TraficSourceID) = -128 ? -127 : toInt8(TraficSourceID), +:-] toInt16(SearchEngineID) = -32768 ? -32767 : toInt16(SearchEngineID), +:-] toValidUTF8(toString(SearchPhrase)), +:-] toInt8(AdvEngineID) = -128 ? -127 : toInt8(AdvEngineID), +:-] toInt8(IsArtifical) = -128 ? -127 : toInt8(IsArtifical), +:-] toInt16(WindowClientWidth) = -32768 ? -32767 : toInt16(WindowClientWidth), +:-] toInt16(WindowClientHeight) = -32768 ? -32767 : toInt16(WindowClientHeight), +:-] toInt16(ClientTimeZone) = -32768 ? -32767 : toInt16(ClientTimeZone), +:-] ClientEventTime, +:-] toInt8(SilverlightVersion1) = -128 ? -127 : toInt8(SilverlightVersion1), +:-] toInt8(SilverlightVersion2) = -128 ? -127 : toInt8(SilverlightVersion2), +:-] toInt32(SilverlightVersion3) = -2147483648 ? -2147483647 : toInt32(SilverlightVersion3), +:-] toInt16(SilverlightVersion4) = -32768 ? -32767 : toInt16(SilverlightVersion4), +:-] toValidUTF8(toString(PageCharset)), +:-] toInt32(CodeVersion) = -2147483648 ? -2147483647 : toInt32(CodeVersion), +:-] toInt8(IsLink) = -128 ? -127 : toInt8(IsLink), +:-] toInt8(IsDownload) = -128 ? -127 : toInt8(IsDownload), +:-] toInt8(IsNotBounce) = -128 ? -127 : toInt8(IsNotBounce), +:-] toInt64(FUniqID) = -9223372036854775808 ? -9223372036854775807 : toInt64(FUniqID), +:-] toValidUTF8(toString(OriginalURL)), +:-] toInt32(HID) = -2147483648 ? -2147483647 : toInt32(HID), +:-] toInt8(IsOldCounter) = -128 ? -127 : toInt8(IsOldCounter), +:-] toInt8(IsEvent) = -128 ? -127 : toInt8(IsEvent), +:-] toInt8(IsParameter) = -128 ? -127 : toInt8(IsParameter), +:-] toInt8(DontCountHits) = -128 ? -127 : toInt8(DontCountHits), +:-] toInt8(WithHash) = -128 ? -127 : toInt8(WithHash), +:-] toValidUTF8(toString(HitColor)), +:-] LocalEventTime, +:-] toInt8(Age) = -128 ? -127 : toInt8(Age), +:-] toInt8(Sex) = -128 ? -127 : toInt8(Sex), +:-] toInt8(Income) = -128 ? -127 : toInt8(Income), +:-] toInt16(Interests) = -32768 ? -32767 : toInt16(Interests), +:-] toInt8(Robotness) = -128 ? -127 : toInt8(Robotness), +:-] toInt32(RemoteIP) = -2147483648 ? -2147483647 : toInt32(RemoteIP), +:-] toInt32(WindowName) = -2147483648 ? -2147483647 : toInt32(WindowName), +:-] toInt32(OpenerName) = -2147483648 ? -2147483647 : toInt32(OpenerName), +:-] toInt16(HistoryLength) = -32768 ? -32767 : toInt16(HistoryLength), +:-] toValidUTF8(toString(BrowserLanguage)), +:-] toValidUTF8(toString(BrowserCountry)), +:-] toValidUTF8(toString(SocialNetwork)), +:-] toValidUTF8(toString(SocialAction)), +:-] toInt16(HTTPError) = -32768 ? -32767 : toInt16(HTTPError), +:-] toInt32(SendTiming) = -2147483648 ? -2147483647 : toInt32(SendTiming), +:-] toInt32(DNSTiming) = -2147483648 ? -2147483647 : toInt32(DNSTiming), +:-] toInt32(ConnectTiming) = -2147483648 ? -2147483647 : toInt32(ConnectTiming), +:-] toInt32(ResponseStartTiming) = -2147483648 ? -2147483647 : toInt32(ResponseStartTiming), +:-] toInt32(ResponseEndTiming) = -2147483648 ? -2147483647 : toInt32(ResponseEndTiming), +:-] toInt32(FetchTiming) = -2147483648 ? -2147483647 : toInt32(FetchTiming), +:-] toInt8(SocialSourceNetworkID) = -128 ? -127 : toInt8(SocialSourceNetworkID), +:-] toValidUTF8(toString(SocialSourcePage)), +:-] toInt64(ParamPrice) = -9223372036854775808 ? -9223372036854775807 : toInt64(ParamPrice), +:-] toValidUTF8(toString(ParamOrderID)), +:-] toValidUTF8(toString(ParamCurrency)), +:-] toInt16(ParamCurrencyID) = -32768 ? -32767 : toInt16(ParamCurrencyID), +:-] toValidUTF8(toString(OpenstatServiceName)), +:-] toValidUTF8(toString(OpenstatCampaignID)), +:-] toValidUTF8(toString(OpenstatAdID)), +:-] toValidUTF8(toString(OpenstatSourceID)), +:-] toValidUTF8(toString(UTMSource)), +:-] toValidUTF8(toString(UTMMedium)), +:-] toValidUTF8(toString(UTMCampaign)), +:-] toValidUTF8(toString(UTMContent)), +:-] toValidUTF8(toString(UTMTerm)), +:-] toValidUTF8(toString(FromTag)), +:-] toInt8(HasGCLID) = -128 ? -127 : toInt8(HasGCLID), +:-] toInt64(RefererHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(RefererHash), +:-] toInt64(URLHash) = -9223372036854775808 ? -9223372036854775807 : toInt64(URLHash), +:-] toInt32(CLID) = -2147483648 ? -2147483647 : toInt32(CLID) +:-] FROM hits_100m_obfuscated +:-] INTO OUTFILE '/home/ubuntu/hits_100m_obfuscated_monetdb.tsv' +:-] FORMAT TSV; + +SELECT + if(toInt64(WatchID) = -9223372036854775808, -9223372036854775807, toInt64(WatchID)), + if(toInt8(JavaEnable) = -128, -127, toInt8(JavaEnable)), + toValidUTF8(toString(Title)), + if(toInt16(GoodEvent) = -32768, -32767, toInt16(GoodEvent)), + EventTime, + EventDate, + if(toInt32(CounterID) = -2147483648, -2147483647, toInt32(CounterID)), + if(toInt32(ClientIP) = -2147483648, -2147483647, toInt32(ClientIP)), + if(toInt32(RegionID) = -2147483648, -2147483647, toInt32(RegionID)), + if(toInt64(UserID) = -9223372036854775808, -9223372036854775807, toInt64(UserID)), + if(toInt8(CounterClass) = -128, -127, toInt8(CounterClass)), + if(toInt8(OS) = -128, -127, toInt8(OS)), + if(toInt8(UserAgent) = -128, -127, toInt8(UserAgent)), + toValidUTF8(toString(URL)), + toValidUTF8(toString(Referer)), + if(toInt8(Refresh) = -128, -127, toInt8(Refresh)), + if(toInt16(RefererCategoryID) = -32768, -32767, toInt16(RefererCategoryID)), + if(toInt32(RefererRegionID) = -2147483648, -2147483647, toInt32(RefererRegionID)), + if(toInt16(URLCategoryID) = -32768, -32767, toInt16(URLCategoryID)), + if(toInt32(URLRegionID) = -2147483648, -2147483647, toInt32(URLRegionID)), + if(toInt16(ResolutionWidth) = -32768, -32767, toInt16(ResolutionWidth)), + if(toInt16(ResolutionHeight) = -32768, -32767, toInt16(ResolutionHeight)), + if(toInt8(ResolutionDepth) = -128, -127, toInt8(ResolutionDepth)), + if(toInt8(FlashMajor) = -128, -127, toInt8(FlashMajor)), + if(toInt8(FlashMinor) = -128, -127, toInt8(FlashMinor)), + toValidUTF8(toString(FlashMinor2)), + if(toInt8(NetMajor) = -128, -127, toInt8(NetMajor)), + if(toInt8(NetMinor) = -128, -127, toInt8(NetMinor)), + if(toInt16(UserAgentMajor) = -32768, -32767, toInt16(UserAgentMajor)), + toValidUTF8(toString(UserAgentMinor)), + if(toInt8(CookieEnable) = -128, -127, toInt8(CookieEnable)), + if(toInt8(JavascriptEnable) = -128, -127, toInt8(JavascriptEnable)), + if(toInt8(IsMobile) = -128, -127, toInt8(IsMobile)), + if(toInt8(MobilePhone) = -128, -127, toInt8(MobilePhone)), + toValidUTF8(toString(MobilePhoneModel)), + toValidUTF8(toString(Params)), + if(toInt32(IPNetworkID) = -2147483648, -2147483647, toInt32(IPNetworkID)), + if(toInt8(TraficSourceID) = -128, -127, toInt8(TraficSourceID)), + if(toInt16(SearchEngineID) = -32768, -32767, toInt16(SearchEngineID)), + toValidUTF8(toString(SearchPhrase)), + if(toInt8(AdvEngineID) = -128, -127, toInt8(AdvEngineID)), + if(toInt8(IsArtifical) = -128, -127, toInt8(IsArtifical)), + if(toInt16(WindowClientWidth) = -32768, -32767, toInt16(WindowClientWidth)), + if(toInt16(WindowClientHeight) = -32768, -32767, toInt16(WindowClientHeight)), + if(toInt16(ClientTimeZone) = -32768, -32767, toInt16(ClientTimeZone)), + ClientEventTime, + if(toInt8(SilverlightVersion1) = -128, -127, toInt8(SilverlightVersion1)), + if(toInt8(SilverlightVersion2) = -128, -127, toInt8(SilverlightVersion2)), + if(toInt32(SilverlightVersion3) = -2147483648, -2147483647, toInt32(SilverlightVersion3)), + if(toInt16(SilverlightVersion4) = -32768, -32767, toInt16(SilverlightVersion4)), + toValidUTF8(toString(PageCharset)), + if(toInt32(CodeVersion) = -2147483648, -2147483647, toInt32(CodeVersion)), + if(toInt8(IsLink) = -128, -127, toInt8(IsLink)), + if(toInt8(IsDownload) = -128, -127, toInt8(IsDownload)), + if(toInt8(IsNotBounce) = -128, -127, toInt8(IsNotBounce)), + if(toInt64(FUniqID) = -9223372036854775808, -9223372036854775807, toInt64(FUniqID)), + toValidUTF8(toString(OriginalURL)), + if(toInt32(HID) = -2147483648, -2147483647, toInt32(HID)), + if(toInt8(IsOldCounter) = -128, -127, toInt8(IsOldCounter)), + if(toInt8(IsEvent) = -128, -127, toInt8(IsEvent)), + if(toInt8(IsParameter) = -128, -127, toInt8(IsParameter)), + if(toInt8(DontCountHits) = -128, -127, toInt8(DontCountHits)), + if(toInt8(WithHash) = -128, -127, toInt8(WithHash)), + toValidUTF8(toString(HitColor)), + LocalEventTime, + if(toInt8(Age) = -128, -127, toInt8(Age)), + if(toInt8(Sex) = -128, -127, toInt8(Sex)), + if(toInt8(Income) = -128, -127, toInt8(Income)), + if(toInt16(Interests) = -32768, -32767, toInt16(Interests)), + if(toInt8(Robotness) = -128, -127, toInt8(Robotness)), + if(toInt32(RemoteIP) = -2147483648, -2147483647, toInt32(RemoteIP)), + if(toInt32(WindowName) = -2147483648, -2147483647, toInt32(WindowName)), + if(toInt32(OpenerName) = -2147483648, -2147483647, toInt32(OpenerName)), + if(toInt16(HistoryLength) = -32768, -32767, toInt16(HistoryLength)), + toValidUTF8(toString(BrowserLanguage)), + toValidUTF8(toString(BrowserCountry)), + toValidUTF8(toString(SocialNetwork)), + toValidUTF8(toString(SocialAction)), + if(toInt16(HTTPError) = -32768, -32767, toInt16(HTTPError)), + if(toInt32(SendTiming) = -2147483648, -2147483647, toInt32(SendTiming)), + if(toInt32(DNSTiming) = -2147483648, -2147483647, toInt32(DNSTiming)), + if(toInt32(ConnectTiming) = -2147483648, -2147483647, toInt32(ConnectTiming)), + if(toInt32(ResponseStartTiming) = -2147483648, -2147483647, toInt32(ResponseStartTiming)), + if(toInt32(ResponseEndTiming) = -2147483648, -2147483647, toInt32(ResponseEndTiming)), + if(toInt32(FetchTiming) = -2147483648, -2147483647, toInt32(FetchTiming)), + if(toInt8(SocialSourceNetworkID) = -128, -127, toInt8(SocialSourceNetworkID)), + toValidUTF8(toString(SocialSourcePage)), + if(toInt64(ParamPrice) = -9223372036854775808, -9223372036854775807, toInt64(ParamPrice)), + toValidUTF8(toString(ParamOrderID)), + toValidUTF8(toString(ParamCurrency)), + if(toInt16(ParamCurrencyID) = -32768, -32767, toInt16(ParamCurrencyID)), + toValidUTF8(toString(OpenstatServiceName)), + toValidUTF8(toString(OpenstatCampaignID)), + toValidUTF8(toString(OpenstatAdID)), + toValidUTF8(toString(OpenstatSourceID)), + toValidUTF8(toString(UTMSource)), + toValidUTF8(toString(UTMMedium)), + toValidUTF8(toString(UTMCampaign)), + toValidUTF8(toString(UTMContent)), + toValidUTF8(toString(UTMTerm)), + toValidUTF8(toString(FromTag)), + if(toInt8(HasGCLID) = -128, -127, toInt8(HasGCLID)), + if(toInt64(RefererHash) = -9223372036854775808, -9223372036854775807, toInt64(RefererHash)), + if(toInt64(URLHash) = -9223372036854775808, -9223372036854775807, toInt64(URLHash)), + if(toInt32(CLID) = -2147483648, -2147483647, toInt32(CLID)) +FROM hits_100m_obfuscated +INTO OUTFILE '/home/ubuntu/hits_100m_obfuscated_monetdb.tsv' +FORMAT TSV + + +100000000 rows in set. Elapsed: 460.329 sec. Processed 100.00 million rows, 74.69 GB (217.24 thousand rows/s., 162.25 MB/s.) + +ip-▉▉▉.us-east-2.compute.internal :) Bye. +ubuntu@ip-▉▉▉:~$ ls -l +total 86156956 +-rwxrwxr-x 1 ubuntu ubuntu 502 Aug 12 22:37 benchmark-new.sh +-rwxrwxr-x 1 ubuntu ubuntu 3676614664 Aug 11 02:05 clickhouse +drwxrwxr-x 4 ubuntu ubuntu 4096 Aug 12 22:36 data +drwxr-x--- 2 ubuntu ubuntu 4096 Aug 12 22:10 dictionaries_lib +drwxr-x--- 2 ubuntu ubuntu 4096 Aug 12 22:10 flags +drwxr-x--- 2 ubuntu ubuntu 4096 Aug 12 22:10 format_schemas +-rw-rw-r-- 1 ubuntu ubuntu 74825746736 Aug 12 22:50 hits_100m_obfuscated_monetdb.tsv +drwxrwxr-x 2 ubuntu ubuntu 4096 Aug 12 22:36 hits_100m_obfuscated_v1 +-rw-rw-r-- 1 ubuntu ubuntu 9722280160 Aug 7 2019 hits_100m_obfuscated_v1.tar.xz +drwxrwxr-x 4 ubuntu ubuntu 4096 Aug 12 22:36 metadata +drwxr-x--- 2 ubuntu ubuntu 4096 Aug 12 22:10 metadata_dropped +drwxr-x--- 2 ubuntu ubuntu 4096 Aug 12 22:10 preprocessed_configs +-rw-rw-r-- 1 ubuntu ubuntu 8074 Aug 12 22:11 queries.sql +-rw-r----- 1 ubuntu ubuntu 58 Aug 12 22:36 status +drwxr-x--- 2 ubuntu ubuntu 4096 Aug 12 22:10 tmp +drwxr-x--- 2 ubuntu ubuntu 4096 Aug 12 22:10 user_files +ubuntu@ip-▉▉▉:~$ sudo nano /etc/apt/sources.list.d/monetdb.list +ubuntu@ip-▉▉▉:~$ wget --output-document=- https://www.monetdb.org/downloads/MonetDB-GPG-KEY | sudo apt-key add - +--2020-08-12 22:51:53-- https://www.monetdb.org/downloads/MonetDB-GPG-KEY +Resolving www.monetdb.org (www.monetdb.org)... 192.16.197.137 +Connecting to www.monetdb.org (www.monetdb.org)|192.16.197.137|:443... connected. +HTTP request sent, awaiting response... 200 OK +Length: 6738 (6.6K) +Saving to: ‘STDOUT’ + +- 100%[========================================================================================================================>] 6.58K --.-KB/s in 0s + +2020-08-12 22:51:53 (2.24 GB/s) - written to stdout [6738/6738] + +OK +ubuntu@ip-▉▉▉:~$ sudo apt update +Hit:1 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic InRelease +Get:2 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic-updates InRelease [88.7 kB] +Get:3 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic-backports InRelease [74.6 kB] +Get:4 http://security.ubuntu.com/ubuntu bionic-security InRelease [88.7 kB] +Get:5 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic/universe amd64 Packages [8570 kB] +Get:6 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic/universe Translation-en [4941 kB] +Get:7 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic/multiverse amd64 Packages [151 kB] +Get:8 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic/multiverse Translation-en [108 kB] +Get:9 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic-updates/main amd64 Packages [1032 kB] +Get:10 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic-updates/universe amd64 Packages [1097 kB] +Get:11 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic-updates/universe Translation-en [342 kB] +Get:12 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic-updates/multiverse amd64 Packages [19.2 kB] +Get:13 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic-updates/multiverse Translation-en [6712 B] +Get:14 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic-backports/main amd64 Packages [7516 B] +Get:15 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic-backports/main Translation-en [4764 B] +Get:16 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic-backports/universe amd64 Packages [7736 B] +Get:17 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic-backports/universe Translation-en [4588 B] +Get:18 http://security.ubuntu.com/ubuntu bionic-security/main amd64 Packages [809 kB] +Get:19 https://dev.monetdb.org/downloads/deb bionic InRelease [4457 B] +Get:20 http://security.ubuntu.com/ubuntu bionic-security/main Translation-en [254 kB] +Get:21 https://dev.monetdb.org/downloads/deb bionic/monetdb amd64 Packages [72.8 kB] +Get:22 http://security.ubuntu.com/ubuntu bionic-security/universe amd64 Packages [689 kB] +Get:23 http://security.ubuntu.com/ubuntu bionic-security/universe Translation-en [228 kB] +Get:24 http://security.ubuntu.com/ubuntu bionic-security/multiverse amd64 Packages [8100 B] +Get:25 http://security.ubuntu.com/ubuntu bionic-security/multiverse Translation-en [2852 B] +Fetched 18.6 MB in 3s (6053 kB/s) +Reading package lists... Done +Building dependency tree +Reading state information... Done +2 packages can be upgraded. Run 'apt list --upgradable' to see them. +ubuntu@ip-▉▉▉:~$ sudo apt install monetdb5-sql monetdb-client +Reading package lists... Done +Building dependency tree +Reading state information... Done +The following additional packages will be installed: + libmonetdb-client12 libmonetdb-stream13 libmonetdb20 monetdb5-server monetdb5-server-hugeint monetdb5-sql-hugeint +The following NEW packages will be installed: + libmonetdb-client12 libmonetdb-stream13 libmonetdb20 monetdb-client monetdb5-server monetdb5-server-hugeint monetdb5-sql monetdb5-sql-hugeint +0 upgraded, 8 newly installed, 0 to remove and 2 not upgraded. +Need to get 3315 kB of archives. +After this operation, 13.6 MB of additional disk space will be used. +Do you want to continue? [Y/n] +Get:1 https://dev.monetdb.org/downloads/deb bionic/monetdb amd64 libmonetdb-client12 amd64 11.37.11 [104 kB] +Get:2 https://dev.monetdb.org/downloads/deb bionic/monetdb amd64 libmonetdb-stream13 amd64 11.37.11 [92.4 kB] +Get:3 https://dev.monetdb.org/downloads/deb bionic/monetdb amd64 libmonetdb20 amd64 11.37.11 [1229 kB] +Get:4 https://dev.monetdb.org/downloads/deb bionic/monetdb amd64 monetdb-client amd64 11.37.11 [149 kB] +Get:5 https://dev.monetdb.org/downloads/deb bionic/monetdb amd64 monetdb5-server amd64 11.37.11 [695 kB] +Get:6 https://dev.monetdb.org/downloads/deb bionic/monetdb amd64 monetdb5-server-hugeint amd64 11.37.11 [92.4 kB] +Get:7 https://dev.monetdb.org/downloads/deb bionic/monetdb amd64 monetdb5-sql amd64 11.37.11 [875 kB] +Get:8 https://dev.monetdb.org/downloads/deb bionic/monetdb amd64 monetdb5-sql-hugeint amd64 11.37.11 [77.7 kB] +Get:8 https://dev.monetdb.org/downloads/deb bionic/monetdb amd64 monetdb5-sql-hugeint amd64 11.37.11 [77.7 kB] +Fetched 3250 kB in 36s (90.3 kB/s) +Selecting previously unselected package libmonetdb-client12. +(Reading database ... 57084 files and directories currently installed.) +Preparing to unpack .../0-libmonetdb-client12_11.37.11_amd64.deb ... +Unpacking libmonetdb-client12 (11.37.11) ... +Selecting previously unselected package libmonetdb-stream13. +Preparing to unpack .../1-libmonetdb-stream13_11.37.11_amd64.deb ... +Unpacking libmonetdb-stream13 (11.37.11) ... +Selecting previously unselected package libmonetdb20. +Preparing to unpack .../2-libmonetdb20_11.37.11_amd64.deb ... +Unpacking libmonetdb20 (11.37.11) ... +Selecting previously unselected package monetdb-client. +Preparing to unpack .../3-monetdb-client_11.37.11_amd64.deb ... +Unpacking monetdb-client (11.37.11) ... +Selecting previously unselected package monetdb5-server. +Preparing to unpack .../4-monetdb5-server_11.37.11_amd64.deb ... +Unpacking monetdb5-server (11.37.11) ... +Selecting previously unselected package monetdb5-server-hugeint. +Preparing to unpack .../5-monetdb5-server-hugeint_11.37.11_amd64.deb ... +Unpacking monetdb5-server-hugeint (11.37.11) ... +Selecting previously unselected package monetdb5-sql. +Preparing to unpack .../6-monetdb5-sql_11.37.11_amd64.deb ... +Unpacking monetdb5-sql (11.37.11) ... +Selecting previously unselected package monetdb5-sql-hugeint. +Preparing to unpack .../7-monetdb5-sql-hugeint_11.37.11_amd64.deb ... +Unpacking monetdb5-sql-hugeint (11.37.11) ... +Setting up libmonetdb20 (11.37.11) ... +Setting up libmonetdb-client12 (11.37.11) ... +Setting up libmonetdb-stream13 (11.37.11) ... +Setting up monetdb-client (11.37.11) ... +Setting up monetdb5-server (11.37.11) ... +Adding group `monetdb' (GID 115) ... +Done. +Warning: The home dir /var/lib/monetdb you specified already exists. +Adding system user `monetdb' (UID 111) ... +Adding new user `monetdb' (UID 111) with group `monetdb' ... +The home directory `/var/lib/monetdb' already exists. Not copying from `/etc/skel'. +adduser: Warning: The home directory `/var/lib/monetdb' does not belong to the user you are currently creating. +Setting up monetdb5-sql (11.37.11) ... +Setting up monetdb5-server-hugeint (11.37.11) ... +Setting up monetdb5-sql-hugeint (11.37.11) ... +Processing triggers for systemd (237-3ubuntu10.42) ... +Processing triggers for man-db (2.8.3-2ubuntu0.1) ... +Processing triggers for ureadahead (0.100.0-21) ... +Processing triggers for libc-bin (2.27-3ubuntu1.2) ... +ubuntu@ip-▉▉▉:~$ sudo systemctl enable monetdbd +Created symlink /etc/systemd/system/multi-user.target.wants/monetdbd.service → /lib/systemd/system/monetdbd.service. +ubuntu@ip-▉▉▉:~$ sudo systemctl start monetdbd +ubuntu@ip-▉▉▉:~$ sudo usermod -a -G monetdb $USER +ubuntu@ip-▉▉▉:~$ logout +Connection to ec2-▉▉▉.us-east-2.compute.amazonaws.com closed. +milovidov@milovidov-desktop:~$ ssh -i ~/.ssh/aws_milovidov.pem ubuntu@ec2-▉▉▉.us-east-2.compute.amazonaws.com +Welcome to Ubuntu 18.04.5 LTS (GNU/Linux 5.3.0-1032-aws x86_64) + + * Documentation: https://help.ubuntu.com + * Management: https://landscape.canonical.com + * Support: https://ubuntu.com/advantage + + System information as of Wed Aug 12 22:53:13 UTC 2020 + + System load: 0.31 Processes: 329 + Usage of /: 50.1% of 193.82GB Users logged in: 1 + Memory usage: 2% IP address for ens5: ▉▉▉ + Swap usage: 0% + + +2 packages can be updated. +2 updates are security updates. + + +Last login: Wed Aug 12 22:11:01 2020 from 109.252.51.97 +ubuntu@ip-▉▉▉:~$ monetdbd create /opt/monetdb +unable to create directory '/opt/monetdb': Permission denied +ubuntu@ip-▉▉▉:~$ sudo monetdbd create /opt/monetdb +ubuntu@ip-▉▉▉:~$ sudo monetdbd start /opt/monetdb +monetdbd: binding to stream socket port 50000 failed: Address already in use +ubuntu@ip-▉▉▉:~$ monetdb create test +monetdb: cannot connect: control socket does not exist +ubuntu@ip-▉▉▉:~$ sudo systemctl start monetdbd +ubuntu@ip-▉▉▉:~$ monetdb create test +monetdb: cannot connect: control socket does not exist +ubuntu@ip-▉▉▉:~$ sudo systemctl start monetdbd +ubuntu@ip-▉▉▉:~$ sudo monetdb create test +monetdb: cannot connect: control socket does not exist +ubuntu@ip-▉▉▉:~$ less /var/log/monetdb/merovingian.log +/var/log/monetdb/merovingian.log: Permission denied +ubuntu@ip-▉▉▉:~$ sudo less /var/log/monetdb/merovingian.log +ubuntu@ip-▉▉▉:~$ monetdb create test +monetdb: cannot connect: control socket does not exist +ubuntu@ip-▉▉▉:~$ sudo systemctl stop monetdbd +ubuntu@ip-▉▉▉:~$ sudo monetdbd start /opt/monetdb +ubuntu@ip-▉▉▉:~$ monetdb create test +monetdb: cannot connect: no permission to access control socket +ubuntu@ip-▉▉▉:~$ sudo monetdb create test +created database in maintenance mode: test +ubuntu@ip-▉▉▉:~$ sudo monetdb release test +taken database out of maintenance mode: test +ubuntu@ip-▉▉▉:~$ mclient -u monetdb -d test +password: +Welcome to mclient, the MonetDB/SQL interactive terminal (Jun2020-SP1) +Database: MonetDB v11.37.11 (Jun2020-SP1), 'mapi:monetdb://ip-▉▉▉:50000/test' +FOLLOW US on https://twitter.com/MonetDB or https://github.com/MonetDB/MonetDB +Type \q to quit, \? for a list of available commands +auto commit mode: on +sql>CREATE TABLE hits +more>( +more> "WatchID" BIGINT, +more> "JavaEnable" TINYINT, +more> "Title" TEXT, +more> "GoodEvent" SMALLINT, +more> "EventTime" TIMESTAMP, +more> "EventDate" Date, +more> "CounterID" INTEGER, +more> "ClientIP" INTEGER, +more> "RegionID" INTEGER, +more> "UserID" BIGINT, +more> "CounterClass" TINYINT, +more> "OS" TINYINT, +more> "UserAgent" TINYINT, +more> "URL" TEXT, +more> "Referer" TEXT, +more> "Refresh" TINYINT, +more> "RefererCategoryID" SMALLINT, +more> "RefererRegionID" INTEGER, +more> "URLCategoryID" SMALLINT, +more> "URLRegionID" INTEGER, +more> "ResolutionWidth" SMALLINT, +more> "ResolutionHeight" SMALLINT, +more> "ResolutionDepth" TINYINT, +more> "FlashMajor" TINYINT, +more> "FlashMinor" TINYINT, +more> "FlashMinor2" TEXT, +more> "NetMajor" TINYINT, +more> "NetMinor" TINYINT, +more> "UserAgentMajor" SMALLINT, +more> "UserAgentMinor" TEXT, +more> "CookieEnable" TINYINT, +more> "JavascriptEnable" TINYINT, +more> "IsMobile" TINYINT, +more> "MobilePhone" TINYINT, +more> "MobilePhoneModel" TEXT, +more> "Params" TEXT, +more> "IPNetworkID" INTEGER, +more> "TraficSourceID" TINYINT, +more> "SearchEngineID" SMALLINT, +more> "SearchPhrase" TEXT, +more> "AdvEngineID" TINYINT, +more> "IsArtifical" TINYINT, +more> "WindowClientWidth" SMALLINT, +more> "WindowClientHeight" SMALLINT, +more> "ClientTimeZone" SMALLINT, +more> "ClientEventTime" TIMESTAMP, +more> "SilverlightVersion1" TINYINT, +more> "SilverlightVersion2" TINYINT, +more> "SilverlightVersion3" INTEGER, +more> "SilverlightVersion4" SMALLINT, +more> "PageCharset" TEXT, +more> "CodeVersion" INTEGER, +more> "IsLink" TINYINT, +more> "IsDownload" TINYINT, +more> "IsNotBounce" TINYINT, +more> "FUniqID" BIGINT, +more> "OriginalURL" TEXT, +more> "HID" INTEGER, +more> "IsOldCounter" TINYINT, +more> "IsEvent" TINYINT, +more> "IsParameter" TINYINT, +more> "DontCountHits" TINYINT, +more> "WithHash" TINYINT, +more> "HitColor" TEXT, +more> "LocalEventTime" TIMESTAMP, +more> "Age" TINYINT, +more> "Sex" TINYINT, +more> "Income" TINYINT, +more> "Interests" SMALLINT, +more> "Robotness" TINYINT, +more> "RemoteIP" INTEGER, +more> "WindowName" INTEGER, +more> "OpenerName" INTEGER, +more> "HistoryLength" SMALLINT, +more> "BrowserLanguage" TEXT, +more> "BrowserCountry" TEXT, +more> "SocialNetwork" TEXT, +more> "SocialAction" TEXT, +more> "HTTPError" SMALLINT, +more> "SendTiming" INTEGER, +more> "DNSTiming" INTEGER, +more> "ConnectTiming" INTEGER, +more> "ResponseStartTiming" INTEGER, +more> "ResponseEndTiming" INTEGER, +more> "FetchTiming" INTEGER, +more> "SocialSourceNetworkID" TINYINT, +more> "SocialSourcePage" TEXT, +more> "ParamPrice" BIGINT, +more> "ParamOrderID" TEXT, +more> "ParamCurrency" TEXT, +more> "ParamCurrencyID" SMALLINT, +more> "OpenstatServiceName" TEXT, +more> "OpenstatCampaignID" TEXT, +more> "OpenstatAdID" TEXT, +more> "OpenstatSourceID" TEXT, +more> "UTMSource" TEXT, +more> "UTMMedium" TEXT, +more> "UTMCampaign" TEXT, +more> "UTMContent" TEXT, +more> "UTMTerm" TEXT, +more> "FromTag" TEXT, +more> "HasGCLID" TINYINT, +more> "RefererHash" BIGINT, +more> "URLHash" BIGINT, +more> "CLID" INTEGER +more>); +operation successful +sql>ubuntu@ip-▉▉▉:~$ mclient -u monetdb -d test +password: +Welcome to mclient, the MonetDB/SQL interactive terminal (Jun2020-SP1) +Database: MonetDB v11.37.11 (Jun2020-SP1), 'mapi:monetdb://ip-▉▉▉:50000/test' +FOLLOW US on https://twitter.com/MonetDB or https://github.com/MonetDB/MonetDB +Type \q to quit, \? for a list of available commands +auto commit mode: on +sql>COPY INTO hits FROM '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv' USING DELIMITERS '\t'; +Cannot open file '/home/milovidov/example_datasets/hits_100m_obfuscated_monetdb.tsv': No such file or directory +sql>COPY INTO hits FROM '/home/ubuntu/hits_100m_obfuscated_monetdb.tsv' USING DELIMITERS '\t'; +100000000 affected rows +sql>SELECT count(*) FROM hits; ++-----------+ +| %1 | ++===========+ +| 100000000 | ++-----------+ +1 tuple +sql>CREATE INDEX hits_idx ON hits ("CounterID", "EventDate"); +operation successful +sql>ubuntu@ip-▉▉▉:~$ wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/monetdb/{queries.sql,benchmark.sh,send-query} +--2020-08-12 23:28:53-- https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/monetdb/queries.sql +Resolving raw.githubusercontent.com (raw.githubusercontent.com)... 151.101.200.133 +Connecting to raw.githubusercontent.com (raw.githubusercontent.com)|151.101.200.133|:443... connected. +HTTP request sent, awaiting response... 200 OK +Length: 8964 (8.8K) [text/plain] +Saving to: ‘queries.sql.1’ + +queries.sql.1 100%[========================================================================================================================>] 8.75K --.-KB/s in 0s + +2020-08-12 23:28:58 (93.2 MB/s) - ‘queries.sql.1’ saved [8964/8964] + +--2020-08-12 23:28:58-- https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/monetdb/benchmark.sh +Reusing existing connection to raw.githubusercontent.com:443. +HTTP request sent, awaiting response... 200 OK +Length: 285 [text/plain] +Saving to: ‘benchmark.sh’ + +benchmark.sh 100%[========================================================================================================================>] 285 --.-KB/s in 0s + +2020-08-12 23:28:58 (26.2 MB/s) - ‘benchmark.sh’ saved [285/285] + +--2020-08-12 23:28:58-- https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/monetdb/send-query +Reusing existing connection to raw.githubusercontent.com:443. +HTTP request sent, awaiting response... 200 OK +Length: 249 [text/plain] +Saving to: ‘send-query’ + +send-query 100%[========================================================================================================================>] 249 --.-KB/s in 0s + +2020-08-12 23:28:58 (24.4 MB/s) - ‘send-query’ saved [249/249] + +FINISHED --2020-08-12 23:28:58-- +Total wall clock time: 5.4s +Downloaded: 3 files, 9.3K in 0s (81.0 MB/s) +ubuntu@ip-▉▉▉:~$ chmod a+x benchmark.sh send-query +ubuntu@ip-▉▉▉:~$ ./benchmark.sh | tee log.txt +3 +SELECT count() FROM hits; +3 +SELECT count() FROM hits WHERE AdvEngineID != 0; +3 +SELECT sum(AdvEngineID), count(), avg(ResolutionWidth) FROM hits ; +3 +SELECT sum(UserID) FROM hits ; +3 +SELECT uniq(UserID) FROM hits ; +3 +SELECT uniq(SearchPhrase) FROM hits ; +3 +SELECT min(EventDate), max(EventDate) FROM hits ; +3 +SELECT AdvEngineID, count() FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count() DESC; +3 +SELECT RegionID, uniq(UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +3 +SELECT RegionID, sum(AdvEngineID), count() AS c, avg(ResolutionWidth), uniq(UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +3 +SELECT MobilePhoneModel, uniq(UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +3 +SELECT MobilePhone, MobilePhoneModel, uniq(UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +3 +SELECT SearchPhrase, count() AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +3 +SELECT SearchPhrase, uniq(UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +3 +SELECT SearchEngineID, SearchPhrase, count() AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +3 +SELECT UserID, count() FROM hits GROUP BY UserID ORDER BY count() DESC LIMIT 10; +3 +SELECT UserID, SearchPhrase, count() FROM hits GROUP BY UserID, SearchPhrase ORDER BY count() DESC LIMIT 10; +3 +SELECT UserID, SearchPhrase, count() FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +3 +SELECT UserID, toMinute(EventTime) AS m, SearchPhrase, count() FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY count() DESC LIMIT 10; +3 +SELECT UserID FROM hits WHERE UserID = 12345678901234567890; +3 +SELECT count() FROM hits WHERE URL LIKE '%metrika%'; +3 +SELECT SearchPhrase, any(URL), count() AS c FROM hits WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +3 +SELECT SearchPhrase, any(URL), any(Title), count() AS c, uniq(UserID) FROM hits WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +3 +SELECT * FROM hits WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; +3 +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +3 +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +3 +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +3 +SELECT CounterID, avg(length(URL)) AS l, count() AS c FROM hits WHERE URL != '' GROUP BY CounterID HAVING c > 100000 ORDER BY l DESC LIMIT 25; +3 +SELECT domainWithoutWWW(Referer) AS key, avg(length(Referer)) AS l, count() AS c, any(Referer) FROM hits WHERE Referer != '' GROUP BY key HAVING c > 100000 ORDER BY l DESC LIMIT 25; +3 +SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits; +3 +SELECT SearchEngineID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +3 +SELECT WatchID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +3 +SELECT WatchID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +3 +SELECT URL, count() AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +3 +SELECT 1, URL, count() AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +3 +SELECT ClientIP AS x, x - 1, x - 2, x - 3, count() AS c FROM hits GROUP BY x, x - 1, x - 2, x - 3 ORDER BY c DESC LIMIT 10; +3 +SELECT URL, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND notEmpty(URL) GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +3 +SELECT Title, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND notEmpty(Title) GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +3 +SELECT URL, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; +3 +SELECT TraficSourceID, SearchEngineID, AdvEngineID, ((SearchEngineID = 0 AND AdvEngineID = 0) ? Referer : '') AS Src, URL AS Dst, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; +3 +SELECT URLHash, EventDate, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = halfMD5('http://example.ru/') GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100; +3 +SELECT WindowClientWidth, WindowClientHeight, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = halfMD5('http://example.ru/') GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; +3 +SELECT toStartOfMinute(EventTime) AS Minute, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; +ubuntu@ip-▉▉▉:~$ mclient -u monetdb -d test +password: +Welcome to mclient, the MonetDB/SQL interactive terminal (Jun2020-SP1) +Database: MonetDB v11.37.11 (Jun2020-SP1), 'mapi:monetdb://ip-▉▉▉:50000/test' +FOLLOW US on https://twitter.com/MonetDB or https://github.com/MonetDB/MonetDB +Type \q to quit, \? for a list of available commands +auto commit mode: on +sql>SELECT COUNT(DISTINCT "SearchPhrase") FROM hits; ++---------+ +| %1 | ++=========+ +| 6019589 | ++---------+ +1 tuple +sql>ubuntu@ip-▉▉▉:~$ expect + +Command 'expect' not found, but can be installed with: + +sudo snap install expect # version 5.45-7snap0, or +sudo apt install expect + +See 'snap info expect' for additional versions. + +ubuntu@ip-▉▉▉:~$ sudo apt install expect +Reading package lists... Done +Building dependency tree +Reading state information... Done +The following additional packages will be installed: + libtcl8.6 tcl-expect tcl8.6 +Suggested packages: + tk8.6 tcl-tclreadline +The following NEW packages will be installed: + expect libtcl8.6 tcl-expect tcl8.6 +0 upgraded, 4 newly installed, 0 to remove and 2 not upgraded. +Need to get 1138 kB of archives. +After this operation, 4598 kB of additional disk space will be used. +Do you want to continue? [Y/n] +Get:1 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic/main amd64 libtcl8.6 amd64 8.6.8+dfsg-3 [881 kB] +Get:2 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic/universe amd64 tcl-expect amd64 5.45.4-1 [105 kB] +Get:3 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic/universe amd64 expect amd64 5.45.4-1 [137 kB] +Get:4 http://us-east-2.ec2.archive.ubuntu.com/ubuntu bionic/main amd64 tcl8.6 amd64 8.6.8+dfsg-3 [14.4 kB] +Fetched 1138 kB in 0s (31.4 MB/s) +Selecting previously unselected package libtcl8.6:amd64. +(Reading database ... 57274 files and directories currently installed.) +Preparing to unpack .../libtcl8.6_8.6.8+dfsg-3_amd64.deb ... +Unpacking libtcl8.6:amd64 (8.6.8+dfsg-3) ... +Selecting previously unselected package tcl-expect:amd64. +Preparing to unpack .../tcl-expect_5.45.4-1_amd64.deb ... +Unpacking tcl-expect:amd64 (5.45.4-1) ... +Selecting previously unselected package expect. +Preparing to unpack .../expect_5.45.4-1_amd64.deb ... +Unpacking expect (5.45.4-1) ... +Selecting previously unselected package tcl8.6. +Preparing to unpack .../tcl8.6_8.6.8+dfsg-3_amd64.deb ... +Unpacking tcl8.6 (8.6.8+dfsg-3) ... +Setting up libtcl8.6:amd64 (8.6.8+dfsg-3) ... +Setting up tcl-expect:amd64 (5.45.4-1) ... +Setting up tcl8.6 (8.6.8+dfsg-3) ... +Setting up expect (5.45.4-1) ... +Processing triggers for man-db (2.8.3-2ubuntu0.1) ... +Processing triggers for libc-bin (2.27-3ubuntu1.2) ... +^[[Aubuntu@ip-▉▉▉:~$ ./benchmark.sh | tee log.txt +3 +SELECT count() FROM hits; +clk: 0.299 ms +clk: 0.380 ms +clk: 0.471 ms +3 +SELECT count() FROM hits WHERE AdvEngineID != 0; +clk: 0.297 ms +clk: 0.342 ms +clk: 0.363 ms +3 +SELECT sum(AdvEngineID), count(), avg(ResolutionWidth) FROM hits ; +SELECT: identifier 'advengineid' unknown +clk: 0.485 ms +SELECT: identifier 'advengineid' unknown +clk: 0.228 ms +SELECT: identifier 'advengineid' unknown +clk: 0.233 ms +3 +SELECT sum(UserID) FROM hits ; +SELECT: identifier 'userid' unknown +clk: 0.239 ms +SELECT: identifier 'userid' unknown +clk: 0.492 ms +SELECT: identifier 'userid' unknown +clk: 0.388 ms +3 +SELECT uniq(UserID) FROM hits ; +SELECT: identifier 'userid' unknown +clk: 0.267 ms +SELECT: identifier 'userid' unknown +clk: 0.222 ms +SELECT: identifier 'userid' unknown +clk: 0.212 ms +3 +SELECT uniq(SearchPhrase) FROM hits ; +SELECT: identifier 'searchphrase' unknown +clk: 0.498 ms +SELECT: identifier 'searchphrase' unknown +clk: 0.255 ms +SELECT: identifier 'searchphrase' unknown +clk: 0.224 ms +3 +SELECT min(EventDate), max(EventDate) FROM hits ; +SELECT: identifier 'eventdate' unknown +clk: 0.272 ms +SELECT: identifier 'eventdate' unknown +clk: 0.205 ms +SELECT: identifier 'eventdate' unknown +clk: 0.209 ms +3 +SELECT AdvEngineID, count() FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count() DESC; +clk: 0.439 ms +clk: 0.185 ms +clk: 0.235 ms +3 +SELECT RegionID, uniq(UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT: identifier 'regionid' unknown +clk: 0.283 ms +SELECT: identifier 'regionid' unknown +clk: 0.207 ms +SELECT: identifier 'regionid' unknown +clk: 0.212 ms +3 +SELECT RegionID, sum(AdvEngineID), count() AS c, avg(ResolutionWidth), uniq(UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT: identifier 'regionid' unknown +clk: 0.271 ms +SELECT: identifier 'regionid' unknown +clk: 0.231 ms +SELECT: identifier 'regionid' unknown +clk: 0.227 ms +3 +SELECT MobilePhoneModel, uniq(UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +clk: 0.256 ms +clk: 0.229 ms +clk: 0.223 ms +3 +SELECT MobilePhone, MobilePhoneModel, uniq(UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +clk: 0.244 ms +clk: 0.201 ms +clk: 0.234 ms +3 +SELECT SearchPhrase, count() AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +clk: 0.223 ms +clk: 0.210 ms +clk: 0.207 ms +3 +SELECT SearchPhrase, uniq(UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +clk: 0.220 ms +clk: 0.231 ms +clk: 0.189 ms +3 +SELECT SearchEngineID, SearchPhrase, count() AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +clk: 0.238 ms +clk: 0.226 ms +clk: 0.221 ms +3 +SELECT UserID, count() FROM hits GROUP BY UserID ORDER BY count() DESC LIMIT 10; +SELECT: identifier 'userid' unknown +clk: 0.216 ms +SELECT: identifier 'userid' unknown +clk: 0.214 ms +SELECT: identifier 'userid' unknown +clk: 0.465 ms +3 +SELECT UserID, SearchPhrase, count() FROM hits GROUP BY UserID, SearchPhrase ORDER BY count() DESC LIMIT 10; +SELECT: identifier 'userid' unknown +clk: 0.250 ms +SELECT: identifier 'userid' unknown +clk: 0.304 ms +SELECT: identifier 'userid' unknown +clk: 0.239 ms +3 +SELECT UserID, SearchPhrase, count() FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT: identifier 'userid' unknown +clk: 0.203 ms +SELECT: identifier 'userid' unknown +clk: 0.209 ms +SELECT: identifier 'userid' unknown +clk: 0.221 ms +3 +SELECT UserID, toMinute(EventTime) AS m, SearchPhrase, count() FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY count() DESC LIMIT 10; +SELECT: identifier 'userid' unknown +clk: 0.238 ms +SELECT: identifier 'userid' unknown +clk: 0.249 ms +SELECT: identifier 'userid' unknown +clk: 0.258 ms +3 +SELECT UserID FROM hits WHERE UserID = 12345678901234567890; +SELECT: identifier 'userid' unknown +clk: 0.204 ms +SELECT: identifier 'userid' unknown +clk: 0.187 ms +SELECT: identifier 'userid' unknown +clk: 0.195 ms +3 +SELECT count() FROM hits WHERE URL LIKE '%metrika%'; +SELECT: identifier 'url' unknown +clk: 0.575 ms +SELECT: identifier 'url' unknown +clk: 0.191 ms +SELECT: identifier 'url' unknown +clk: 0.191 ms +3 +SELECT SearchPhrase, any(URL), count() AS c FROM hits WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +syntax error, unexpected ANY in: "select searchphrase, any" +clk: 0.411 ms +syntax error, unexpected ANY in: "select searchphrase, any" +clk: 0.233 ms +syntax error, unexpected ANY in: "select searchphrase, any" +clk: 0.208 ms +3 +SELECT SearchPhrase, any(URL), any(Title), count() AS c, uniq(UserID) FROM hits WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +syntax error, unexpected ANY in: "select searchphrase, any" +clk: 0.266 ms +syntax error, unexpected ANY in: "select searchphrase, any" +clk: 0.238 ms +syntax error, unexpected ANY in: "select searchphrase, any" +clk: 0.311 ms +3 +SELECT * FROM hits WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; +SELECT: identifier 'url' unknown +clk: 0.221 ms +SELECT: identifier 'url' unknown +clk: 0.205 ms +SELECT: identifier 'url' unknown +clk: 0.201 ms +3 +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +clk: 0.183 ms +clk: 0.539 ms +clk: 0.183 ms +3 +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +clk: 0.227 ms +clk: 0.187 ms +clk: 0.201 ms +3 +SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +clk: 0.214 ms +clk: 0.192 ms +clk: 0.230 ms +3 +SELECT CounterID, avg(length(URL)) AS l, count() AS c FROM hits WHERE URL != '' GROUP BY CounterID HAVING c > 100000 ORDER BY l DESC LIMIT 25; +clk: 0.245 ms +clk: 0.244 ms +clk: 0.216 ms +3 +SELECT domainWithoutWWW(Referer) AS key, avg(length(Referer)) AS l, count() AS c, any(Referer) FROM hits WHERE Referer != '' GROUP BY key HAVING c > 100000 ORDER BY l DESC LIMIT 25; +syntax error, unexpected ANY in: "select domainwithoutwww(referer) as key, avg(length(referer)) as l, count() as c" +clk: 0.415 ms +syntax error, unexpected ANY in: "select domainwithoutwww(referer) as key, avg(length(referer)) as l, count() as c" +clk: 0.221 ms +syntax error, unexpected ANY in: "select domainwithoutwww(referer) as key, avg(length(referer)) as l, count() as c" +clk: 0.232 ms +3 +SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits; +SELECT: identifier 'resolutionwidth' unknown +clk: 0.869 ms +SELECT: identifier 'resolutionwidth' unknown +clk: 0.705 ms +SELECT: identifier 'resolutionwidth' unknown +clk: 0.780 ms +3 +SELECT SearchEngineID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +clk: 0.251 ms +clk: 0.239 ms +clk: 0.231 ms +3 +SELECT WatchID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +clk: 0.237 ms +clk: 0.234 ms +clk: 0.209 ms +3 +SELECT WatchID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT: identifier 'watchid' unknown +clk: 0.252 ms +SELECT: identifier 'watchid' unknown +clk: 0.261 ms +SELECT: identifier 'watchid' unknown +clk: 0.226 ms +3 +SELECT URL, count() AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT: identifier 'url' unknown +clk: 0.459 ms +SELECT: identifier 'url' unknown +clk: 0.195 ms +SELECT: identifier 'url' unknown +clk: 0.206 ms +3 +SELECT 1, URL, count() AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +SELECT: identifier 'url' unknown +clk: 0.254 ms +SELECT: identifier 'url' unknown +clk: 0.204 ms +SELECT: identifier 'url' unknown +clk: 0.238 ms +3 +SELECT ClientIP AS x, x - 1, x - 2, x - 3, count() AS c FROM hits GROUP BY x, x - 1, x - 2, x - 3 ORDER BY c DESC LIMIT 10; +SELECT: identifier 'clientip' unknown +clk: 0.275 ms +SELECT: identifier 'clientip' unknown +clk: 0.254 ms +SELECT: identifier 'clientip' unknown +clk: 0.230 ms +3 +SELECT URL, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND notEmpty(URL) GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT: identifier 'counterid' unknown +clk: 0.446 ms +SELECT: identifier 'counterid' unknown +clk: 0.276 ms +SELECT: identifier 'counterid' unknown +clk: 0.255 ms +3 +SELECT Title, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND notEmpty(Title) GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT: identifier 'counterid' unknown +clk: 0.248 ms +SELECT: identifier 'counterid' unknown +clk: 0.237 ms +SELECT: identifier 'counterid' unknown +clk: 0.294 ms +3 +SELECT URL, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; +SELECT: identifier 'counterid' unknown +clk: 0.302 ms +SELECT: identifier 'counterid' unknown +clk: 0.234 ms +SELECT: identifier 'counterid' unknown +clk: 0.274 ms +3 +SELECT TraficSourceID, SearchEngineID, AdvEngineID, ((SearchEngineID = 0 AND AdvEngineID = 0) ? Referer : '') AS Src, URL AS Dst, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; +syntax error, unexpected '?', expecting ')' or OR in: "select traficsourceid, searchengineid, advengineid, ((searchengineid = 0 and adv" +clk: 0.446 ms +syntax error, unexpected '?', expecting ')' or OR in: "select traficsourceid, searchengineid, advengineid, ((searchengineid = 0 and adv" +clk: 0.239 ms +syntax error, unexpected '?', expecting ')' or OR in: "select traficsourceid, searchengineid, advengineid, ((searchengineid = 0 and adv" +clk: 0.232 ms +3 +SELECT URLHash, EventDate, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = halfMD5('http://example.ru/') GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100; +SELECT: identifier 'counterid' unknown +clk: 0.259 ms +SELECT: identifier 'counterid' unknown +clk: 0.263 ms +SELECT: identifier 'counterid' unknown +clk: 0.267 ms +3 +SELECT WindowClientWidth, WindowClientHeight, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = halfMD5('http://example.ru/') GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; +SELECT: identifier 'counterid' unknown +clk: 0.253 ms +SELECT: identifier 'counterid' unknown +clk: 0.281 ms +SELECT: identifier 'counterid' unknown +clk: 0.556 ms +3 +SELECT toStartOfMinute(EventTime) AS Minute, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; +syntax error, unexpected MINUTE, expecting SCOLON in: "select tostartofminute(eventtime) as minute" +clk: 0.239 ms +syntax error, unexpected MINUTE, expecting SCOLON in: "select tostartofminute(eventtime) as minute" +clk: 0.228 ms +syntax error, unexpected MINUTE, expecting SCOLON in: "select tostartofminute(eventtime) as minute" +clk: 0.223 ms +ubuntu@ip-▉▉▉:~$ less queries.sql +ubuntu@ip-▉▉▉:~$ rm queries.sql +ubuntu@ip-▉▉▉:~$ mv queries.sql.1 queries.sql +ubuntu@ip-▉▉▉:~$ ./benchmark.sh | tee log.txt +3 +SELECT count(*) FROM hits; +1 tuple +clk: 2.346 ms +1 tuple +clk: 1.873 ms +1 tuple +clk: 1.862 ms +3 +SELECT count(*) FROM hits WHERE "AdvEngineID" <> 0; +1 tuple +clk: 1.137 sec +1 tuple +clk: 36.303 ms +1 tuple +clk: 35.399 ms +3 +SELECT sum("AdvEngineID"), count(*), avg("ResolutionWidth") FROM hits; +1 tuple +clk: 194.816 ms +1 tuple +clk: 54.757 ms +1 tuple +clk: 51.177 ms +3 +SELECT sum("UserID") FROM hits; +overflow in calculation. +clk: 7.495 ms +overflow in calculation. +clk: 2.967 ms +overflow in calculation. +clk: 3.073 ms +3 +SELECT COUNT(DISTINCT "UserID") FROM hits; +1 tuple +clk: 5.556 sec +1 tuple +clk: 3.550 sec +1 tuple +clk: 3.797 sec +3 +SELECT COUNT(DISTINCT "SearchPhrase") FROM hits; +1 tuple +clk: 6.228 sec +1 tuple +clk: 4.042 sec +1 tuple +clk: 3.967 sec +3 +SELECT min("EventDate"), max("EventDate") FROM hits; +1 tuple +clk: 67.704 ms +1 tuple +clk: 50.072 ms +1 tuple +clk: 51.354 ms +3 +SELECT "AdvEngineID", count(*) FROM hits WHERE "AdvEngineID" <> 0 GROUP BY "AdvEngineID" ORDER BY count(*) DESC; +18 tuples +clk: 93.070 ms +18 tuples +clk: 37.996 ms +18 tuples +clk: 31.543 ms +3 +SELECT "RegionID", COUNT(DISTINCT "UserID") AS u FROM hits GROUP BY "RegionID" ORDER BY u DESC LIMIT 10; +10 tuples +clk: 8.111 sec +10 tuples +clk: 4.839 sec +10 tuples +clk: 5.105 sec +3 +SELECT "RegionID", sum("AdvEngineID"), count(*) AS c, avg("ResolutionWidth"), COUNT(DISTINCT "UserID") FROM hits GROUP BY "RegionID" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 11.571 sec +10 tuples +clk: 7.454 sec +10 tuples +clk: 7.433 sec +3 +SELECT "MobilePhoneModel", COUNT(DISTINCT "UserID") AS u FROM hits WHERE "MobilePhoneModel" <> '' GROUP BY "MobilePhoneModel" ORDER BY u DESC LIMIT 10; +10 tuples +clk: 3.080 sec +10 tuples +clk: 387.757 ms +10 tuples +clk: 348.083 ms +3 +SELECT "MobilePhone", "MobilePhoneModel", COUNT(DISTINCT "UserID") AS u FROM hits WHERE "MobilePhoneModel" <> '' GROUP BY "MobilePhone", "MobilePhoneModel" ORDER BY u DESC LIMIT 10; +10 tuples +clk: 3.456 sec +10 tuples +clk: 438.178 ms +10 tuples +clk: 393.357 ms +3 +SELECT "SearchPhrase", count(*) AS c FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 8.111 sec +10 tuples +clk: 5.833 sec +10 tuples +clk: 5.808 sec +3 +SELECT "SearchPhrase", COUNT(DISTINCT "UserID") AS u FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY u DESC LIMIT 10; +10 tuples +clk: 16.087 sec +10 tuples +clk: 10.259 sec +10 tuples +clk: 10.159 sec +3 +SELECT "SearchEngineID", "SearchPhrase", count(*) AS c FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchEngineID", "SearchPhrase" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 9.149 sec +10 tuples +clk: 6.234 sec +10 tuples +clk: 6.197 sec +3 +SELECT "UserID", count(*) FROM hits GROUP BY "UserID" ORDER BY count(*) DESC LIMIT 10; +10 tuples +clk: 5.809 sec +10 tuples +clk: 4.814 sec +10 tuples +clk: 4.413 sec +3 +SELECT "UserID", "SearchPhrase", count(*) FROM hits GROUP BY "UserID", "SearchPhrase" ORDER BY count(*) DESC LIMIT 10; +10 tuples +clk: 15.222 sec +10 tuples +clk: 11.985 sec +10 tuples +clk: 10.959 sec +3 +SELECT "UserID", "SearchPhrase", count(*) FROM hits GROUP BY "UserID", "SearchPhrase" LIMIT 10; +10 tuples +clk: 16.143 sec +10 tuples +clk: 10.903 sec +10 tuples +clk: 11.900 sec +3 +SELECT "UserID", extract(minute FROM "EventTime") AS m, "SearchPhrase", count(*) FROM hits GROUP BY "UserID", m, "SearchPhrase" ORDER BY count(*) DESC LIMIT 10; +10 tuples +clk: 28.492 sec +10 tuples +clk: 22.938 sec +10 tuples +clk: 20.756 sec +3 +SELECT "UserID" FROM hits WHERE "UserID" = -6101065172474983726; +0 tuples +clk: 5.792 sec +0 tuples +clk: 1.764 sec +0 tuples +clk: 0.574 ms +3 +SELECT count(*) FROM hits WHERE "URL" LIKE '%metrika%'; +1 tuple +clk: 22.995 sec +1 tuple +clk: 1.745 sec +1 tuple +clk: 1.597 sec +3 +SELECT "SearchPhrase", min("URL"), count(*) AS c FROM hits WHERE "URL" LIKE '%metrika%' AND "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 44.995 sec +10 tuples +clk: 14.185 sec +10 tuples +clk: 15.712 sec +3 +SELECT "SearchPhrase", min("URL"), min("Title"), count(*) AS c, COUNT(DISTINCT "UserID") FROM hits WHERE "Title" LIKE '%Яндекс%' AND "URL" NOT LIKE '%.yandex.%' AND "SearchPhrase" <> '' GROUP BY "SearchPhrase" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 44.542 sec +10 tuples +clk: 1.850 sec +10 tuples +clk: 1.552 sec +3 +SELECT * FROM hits WHERE "URL" LIKE '%metrika%' ORDER BY "EventTime" LIMIT 10; +10 tuples !85 columns dropped, 29 fields truncated! +clk: 29.023 sec +10 tuples !85 columns dropped, 29 fields truncated! +clk: 1.696 sec +10 tuples !85 columns dropped, 29 fields truncated! +clk: 1.459 sec +3 +SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "EventTime" LIMIT 10; +10 tuples +clk: 5.979 sec +10 tuples +clk: 319.889 ms +10 tuples +clk: 288.515 ms +3 +SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "SearchPhrase" LIMIT 10; +10 tuples +clk: 3.500 sec +10 tuples +clk: 860.218 ms +10 tuples +clk: 852.812 ms +3 +SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "EventTime", "SearchPhrase" LIMIT 10; +10 tuples +clk: 6.540 sec +10 tuples +clk: 853.858 ms +10 tuples +clk: 765.395 ms +3 +SELECT "CounterID", avg(length("URL")) AS l, count(*) AS c FROM hits WHERE "URL" <> '' GROUP BY "CounterID" HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; +25 tuples +clk: 24.407 sec +25 tuples +clk: 2.111 sec +25 tuples +clk: 2.085 sec +3 +SELECT sys.getdomain("Referer") AS key, avg(length("Referer")) AS l, count(*) AS c, min("Referer") FROM hits WHERE "Referer" <> '' GROUP BY key HAVING count(*) > 100000 ORDER BY l DESC LIMIT 25; +clk: 16.458 sec +clk: 7.022 sec +clk: 8.084 sec +3 +SELECT sum("ResolutionWidth"), sum("ResolutionWidth" + 1), sum("ResolutionWidth" + 2), sum("ResolutionWidth" + 3), sum("ResolutionWidth" + 4), sum("ResolutionWidth" + 5), sum("ResolutionWidth" + 6), sum("ResolutionWidth" + 7), sum("ResolutionWidth" + 8), sum("ResolutionWidth" + 9), sum("ResolutionWidth" + 10), sum("ResolutionWidth" + 11), sum("ResolutionWidth" + 12), sum("ResolutionWidth" + 13), sum("ResolutionWidth" + 14), sum("ResolutionWidth" + 15), sum("ResolutionWidth" + 16), sum("ResolutionWidth" + 17), sum("ResolutionWidth" + 18), sum("ResolutionWidth" + 19), sum("ResolutionWidth" + 20), sum("ResolutionWidth" + 21), sum("ResolutionWidth" + 22), sum("ResolutionWidth" + 23), sum("ResolutionWidth" + 24), sum("ResolutionWidth" + 25), sum("ResolutionWidth" + 26), sum("ResolutionWidth" + 27), sum("ResolutionWidth" + 28), sum("ResolutionWidth" + 29), sum("ResolutionWidth" + 30), sum("ResolutionWidth" + 31), sum("ResolutionWidth" + 32), sum("ResolutionWidth" + 33), sum("ResolutionWidth" + 34), sum("ResolutionWidth" + 35), sum("ResolutionWidth" + 36), sum("ResolutionWidth" + 37), sum("ResolutionWidth" + 38), sum("ResolutionWidth" + 39), sum("ResolutionWidth" + 40), sum("ResolutionWidth" + 41), sum("ResolutionWidth" + 42), sum("ResolutionWidth" + 43), sum("ResolutionWidth" + 44), sum("ResolutionWidth" + 45), sum("ResolutionWidth" + 46), sum("ResolutionWidth" + 47), sum("ResolutionWidth" + 48), sum("ResolutionWidth" + 49), sum("ResolutionWidth" + 50), sum("ResolutionWidth" + 51), sum("ResolutionWidth" + 52), sum("ResolutionWidth" + 53), sum("ResolutionWidth" + 54), sum("ResolutionWidth" + 55), sum("ResolutionWidth" + 56), sum("ResolutionWidth" + 57), sum("ResolutionWidth" + 58), sum("ResolutionWidth" + 59), sum("ResolutionWidth" + 60), sum("ResolutionWidth" + 61), sum("ResolutionWidth" + 62), sum("ResolutionWidth" + 63), sum("ResolutionWidth" + 64), sum("ResolutionWidth" + 65), sum("ResolutionWidth" + 66), sum("ResolutionWidth" + 67), sum("ResolutionWidth" + 68), sum("ResolutionWidth" + 69), sum("ResolutionWidth" + 70), sum("ResolutionWidth" + 71), sum("ResolutionWidth" + 72), sum("ResolutionWidth" + 73), sum("ResolutionWidth" + 74), sum("ResolutionWidth" + 75), sum("ResolutionWidth" + 76), sum("ResolutionWidth" + 77), sum("ResolutionWidth" + 78), sum("ResolutionWidth" + 79), sum("ResolutionWidth" + 80), sum("ResolutionWidth" + 81), sum("ResolutionWidth" + 82), sum("ResolutionWidth" + 83), sum("ResolutionWidth" + 84), sum("ResolutionWidth" + 85), sum("ResolutionWidth" + 86), sum("ResolutionWidth" + 87), sum("ResolutionWidth" + 88), sum("ResolutionWidth" + 89) FROM hits; +1 tuple !76 columns dropped! +clk: 3.026 sec +1 tuple !76 columns dropped! +clk: 2.862 sec +1 tuple !76 columns dropped! +clk: 2.846 sec +3 +SELECT "SearchEngineID", "ClientIP", count(*) AS c, sum("Refresh"), avg("ResolutionWidth") FROM hits WHERE "SearchPhrase" <> '' GROUP BY "SearchEngineID", "ClientIP" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 9.145 sec +10 tuples +clk: 3.225 sec +10 tuples +clk: 3.061 sec +3 +SELECT "WatchID", "ClientIP", count(*) AS c, sum("Refresh"), avg("ResolutionWidth") FROM hits WHERE "SearchPhrase" <> '' GROUP BY "WatchID", "ClientIP" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 12.443 sec +10 tuples +clk: 4.298 sec +10 tuples +clk: 4.308 sec +3 +SELECT "WatchID", "ClientIP", count(*) AS c, sum("Refresh"), avg("ResolutionWidth") FROM hits GROUP BY "WatchID", "ClientIP" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 28.515 sec +10 tuples +clk: 27.195 sec +10 tuples +clk: 27.687 sec +3 +SELECT "URL", count(*) AS c FROM hits GROUP BY "URL" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 42.507 sec +10 tuples +clk: 22.997 sec +10 tuples +clk: 21.922 sec +3 +SELECT 1, "URL", count(*) AS c FROM hits GROUP BY 1, "URL" ORDER BY c DESC LIMIT 10; +10 tuples +clk: 42.009 sec +10 tuples +clk: 22.347 sec +10 tuples +clk: 21.288 sec +3 +SELECT "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3, count(*) AS c FROM hits GROUP BY "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3 ORDER BY c DESC LIMIT 10; +10 tuples +clk: 16.540 sec +10 tuples +clk: 14.959 sec +10 tuples +clk: 14.100 sec +3 +SELECT "URL", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "DontCountHits" = 0 AND "Refresh" = 0 AND "URL" <> '' GROUP BY "URL" ORDER BY "PageViews" DESC LIMIT 10; +10 tuples +clk: 2:08 min +10 tuples +clk: 24.004 sec +10 tuples +clk: 23.890 sec +3 +SELECT "Title", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "DontCountHits" = 0 AND "Refresh" = 0 AND "Title" <> '' GROUP BY "Title" ORDER BY "PageViews" DESC LIMIT 10; +10 tuples +clk: 1.893 sec +10 tuples +clk: 245.259 ms +10 tuples +clk: 226.849 ms +3 +SELECT "URL", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 AND "IsLink" <> 0 AND "IsDownload" = 0 GROUP BY "URL" ORDER BY "PageViews" DESC LIMIT 1000; +1000 tuples +clk: 2:09 min +1000 tuples +clk: 22.917 sec +1000 tuples +clk: 23.099 sec +3 +SELECT "TraficSourceID", "SearchEngineID", "AdvEngineID", CASE WHEN ("SearchEngineID" = 0 AND "AdvEngineID" = 0) THEN "Referer" ELSE '' END AS Src, "URL" AS Dst, count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 GROUP BY "TraficSourceID", "SearchEngineID", "AdvEngineID", CASE WHEN ("SearchEngineID" = 0 AND "AdvEngineID" = 0) THEN "Referer" ELSE '' END, "URL" ORDER BY "PageViews" DESC LIMIT 1000; +1000 tuples +clk: 2:09 min +1000 tuples +clk: 23.975 sec +1000 tuples +clk: 22.763 sec +3 +SELECT "URLHash", "EventDate", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 AND "TraficSourceID" IN (-1, 6) AND "RefererHash" = 686716256552154761 GROUP BY "URLHash", "EventDate" ORDER BY "PageViews" DESC LIMIT 100; +0 tuples +clk: 1.148 sec +0 tuples +clk: 114.740 ms +0 tuples +clk: 114.258 ms +3 +SELECT "WindowClientWidth", "WindowClientHeight", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-31' AND "Refresh" = 0 AND "DontCountHits" = 0 AND "URLHash" = 686716256552154761 GROUP BY "WindowClientWidth", "WindowClientHeight" ORDER BY "PageViews" DESC LIMIT 10000; +0 tuples +clk: 2.079 sec +0 tuples +clk: 235.220 ms +0 tuples +clk: 163.850 ms +3 +SELECT DATE_TRUNC('minute', "EventTime") AS "Minute", count(*) AS "PageViews" FROM hits WHERE "CounterID" = 62 AND "EventDate" >= '2013-07-01' AND "EventDate" <= '2013-07-02' AND "Refresh" = 0 AND "DontCountHits" = 0 GROUP BY DATE_TRUNC('minute', "EventTime") ORDER BY DATE_TRUNC('minute', "EventTime"); +0 tuples +clk: 1.690 sec +0 tuples +clk: 148.379 ms +0 tuples +clk: 166.565 ms +ubuntu@ip-▉▉▉:~$ grep clk log.txt | tr -d '\r' | awk '{ if ($3 == "ms") { print $2 / 1000; } else if ($3 == "sec") { print $2 } else { print } }' > tmp.txt +ubuntu@ip-▉▉▉:~$ nano tmp.txt +ubuntu@ip-▉▉▉:~$ awk '{ +> if (i % 3 == 0) { a = $1 } +> else if (i % 3 == 1) { b = $1 } +> else if (i % 3 == 2) { c = $1; print "[" a ", " b ", " c "]," }; +> ++i; }' < tmp.txt +[0.002346, 0.001873, 0.001862], +[1.137, 0.036303, 0.035399], +[0.194816, 0.054757, 0.051177], +[0.007495, 0.002967, 0.003073], +[5.556, 3.550, 3.797], +[6.228, 4.042, 3.967], +[0.067704, 0.050072, 0.051354], +[0.09307, 0.037996, 0.031543], +[8.111, 4.839, 5.105], +[11.571, 7.454, 7.433], +[3.080, 0.387757, 0.348083], +[3.456, 0.438178, 0.393357], +[8.111, 5.833, 5.808], +[16.087, 10.259, 10.159], +[9.149, 6.234, 6.197], +[5.809, 4.814, 4.413], +[15.222, 11.985, 10.959], +[16.143, 10.903, 11.900], +[28.492, 22.938, 20.756], +[5.792, 1.764, 0.000574], +[22.995, 1.745, 1.597], +[44.995, 14.185, 15.712], +[44.542, 1.850, 1.552], +[29.023, 1.696, 1.459], +[5.979, 0.319889, 0.288515], +[3.500, 0.860218, 0.852812], +[6.540, 0.853858, 0.765395], +[24.407, 2.111, 2.085], +[16.458, 7.022, 8.084], +[3.026, 2.862, 2.846], +[9.145, 3.225, 3.061], +[12.443, 4.298, 4.308], +[28.515, 27.195, 27.687], +[42.507, 22.997, 21.922], +[42.009, 22.347, 21.288], +[16.540, 14.959, 14.100], +[128, 24.004, 23.890], +[1.893, 0.245259, 0.226849], +[129, 22.917, 23.099], +[129, 23.975, 22.763], +[1.148, 0.11474, 0.114258], +[2.079, 0.23522, 0.16385], +[1.690, 0.148379, 0.166565], +ubuntu@ip-▉▉▉:~$ From 55c70e6ecbe7d4dcb9a63169bc213b7183cb3085 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov <vzakaznikov@protonmail.com> Date: Thu, 13 Aug 2020 07:40:11 -0400 Subject: [PATCH 301/374] Fixing test clickhouse/ldap authentication/user authentications/rbac=True/login after user cn changed in ldap. --- tests/testflows/ldap/tests/authentications.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/testflows/ldap/tests/authentications.py b/tests/testflows/ldap/tests/authentications.py index c56c23d6813..a1fb27bd51a 100644 --- a/tests/testflows/ldap/tests/authentications.py +++ b/tests/testflows/ldap/tests/authentications.py @@ -185,6 +185,7 @@ def login_after_user_cn_changed_in_ldap(self, server, rbac=False): """Check that login fails after user cn is changed in LDAP.""" self.context.ldap_node = self.context.cluster.node(server) user = None + new_user = None try: with Given(f"I add user to LDAP"): @@ -196,7 +197,7 @@ def login_after_user_cn_changed_in_ldap(self, server, rbac=False): login_and_execute_query(username=user["cn"], password=user["userpassword"]) with When("I change user password in LDAP"): - change_user_cn_in_ldap(user, "myuser2") + new_user = change_user_cn_in_ldap(user, "myuser2") with Then("when I try to login again it should fail"): login_and_execute_query(username=user["cn"], password=user["userpassword"], @@ -205,8 +206,8 @@ def login_after_user_cn_changed_in_ldap(self, server, rbac=False): ) finally: with Finally("I make sure LDAP user is deleted"): - if user is not None: - delete_user_from_ldap(user, exitcode=None) + if new_user is not None: + delete_user_from_ldap(new_user, exitcode=None) @TestScenario @Requirements( From 64e9095b9acbeefacbefca3f850bb97765c6a010 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov <vzakaznikov@protonmail.com> Date: Thu, 13 Aug 2020 09:54:12 -0400 Subject: [PATCH 302/374] Fixing issue with CREATE and DROP USER command by quoting user argument. --- tests/testflows/ldap/tests/common.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/testflows/ldap/tests/common.py b/tests/testflows/ldap/tests/common.py index 0d04818131e..eb21923e930 100644 --- a/tests/testflows/ldap/tests/common.py +++ b/tests/testflows/ldap/tests/common.py @@ -179,13 +179,13 @@ def add_users_identified_with_ldap(*users): try: with Given("I create users"): for user in users: - node.query(f"CREATE USER {user['username']} IDENTIFIED WITH ldap_server BY '{user['server']}'") + node.query(f"CREATE USER '{user['username']}' IDENTIFIED WITH ldap_server BY '{user['server']}'") yield finally: with Finally("I remove users"): for user in users: with By(f"dropping user {user['username']}", flags=TE): - node.query(f"DROP USER IF EXISTS {user['username']}") + node.query(f"DROP USER IF EXISTS '{user['username']}'") @contextmanager def ldap_authenticated_users(*users, config_d_dir="/etc/clickhouse-server/users.d", From 3b92692458f4c3686d26efbc162a52f1db9bb790 Mon Sep 17 00:00:00 2001 From: Amos Bird <amosbird@gmail.com> Date: Thu, 13 Aug 2020 22:22:04 +0800 Subject: [PATCH 303/374] Make drop table a little bit more robust --- src/Storages/MergeTree/MergeTreeData.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1f05476a684..fbf5f1424ef 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1252,7 +1252,16 @@ void MergeTreeData::dropAllData() clearPartsFromFilesystem(all_parts); for (const auto & [path, disk] : getRelativeDataPathsWithDisks()) - disk->removeRecursive(path); + { + try + { + disk->removeRecursive(path); + } + catch (const Poco::FileNotFoundException &) + { + /// If the file is already deleted, do nothing. + } + } LOG_TRACE(log, "dropAllData: done."); } From bcd29dc9f8a6d49afb4a91339ebd76d78a5544aa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu, 13 Aug 2020 18:41:26 +0300 Subject: [PATCH 304/374] Fix typo due to copy-paste --- src/Storages/MergeTree/MergeTreeSettings.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 81142655ab4..7f537ec330a 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -97,7 +97,7 @@ void MergeTreeSettings::sanityCheck(const Settings & query_settings) const " is greater or equals to the value of 'background_pool_size'" " ({}) (the value is defined in users.xml for default profile)." " This indicates incorrect configuration because the maximum size of merge will be always lowered.", - number_of_free_entries_in_pool_to_execute_mutation, + number_of_free_entries_in_pool_to_lower_max_size_of_merge, query_settings.background_pool_size); } } From 0f1c4bc9c442a9c5954d9821e259f056dfa8b723 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov <avtokmakov@yandex-team.ru> Date: Thu, 13 Aug 2020 20:09:40 +0300 Subject: [PATCH 305/374] attach all system tables --- src/Databases/DatabaseAtomic.cpp | 2 + .../System/StorageSystemDetachedParts.cpp | 122 +++++++----------- .../System/StorageSystemDetachedParts.h | 29 ++++- src/Storages/System/attachSystemTables.cpp | 28 ++-- 4 files changed, 94 insertions(+), 87 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 227f4b661db..9cec8179837 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -366,6 +366,8 @@ void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore std::lock_guard lock{mutex}; table_names = table_name_to_path; } + + Poco::File(path_to_table_symlinks).createDirectories(); for (const auto & table : table_names) tryCreateSymlink(table.first, table.second); } diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index b0953ac75e1..bbe7541e3a7 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -12,85 +12,63 @@ namespace DB { -/** - * Implements system table 'detached_parts' which allows to get information - * about detached data parts for tables of MergeTree family. - * We don't use StorageSystemPartsBase, because it introduces virtual _state - * column and column aliases which we don't need. - */ -class StorageSystemDetachedParts final : - public ext::shared_ptr_helper<StorageSystemDetachedParts>, - public IStorage +StorageSystemDetachedParts::StorageSystemDetachedParts(const StorageID & table_id_) + : IStorage(table_id_) { - friend struct ext::shared_ptr_helper<StorageSystemDetachedParts>; -public: - std::string getName() const override { return "SystemDetachedParts"; } + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription{{ + {"database", std::make_shared<DataTypeString>()}, + {"table", std::make_shared<DataTypeString>()}, + {"partition_id", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())}, + {"name", std::make_shared<DataTypeString>()}, + {"disk", std::make_shared<DataTypeString>()}, + {"reason", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())}, + {"min_block_number", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt64>())}, + {"max_block_number", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt64>())}, + {"level", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt32>())} + }}); + setInMemoryMetadata(storage_metadata); +} -protected: - explicit StorageSystemDetachedParts(const StorageID & table_id_) - : IStorage(table_id_) +Pipes StorageSystemDetachedParts::read( + const Names & /* column_names */, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum /*processed_stage*/, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) +{ + StoragesInfoStream stream(query_info, context); + + /// Create the result. + Block block = metadata_snapshot->getSampleBlock(); + MutableColumns new_columns = block.cloneEmptyColumns(); + + while (StoragesInfo info = stream.next()) { - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(ColumnsDescription{{ - {"database", std::make_shared<DataTypeString>()}, - {"table", std::make_shared<DataTypeString>()}, - {"partition_id", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())}, - {"name", std::make_shared<DataTypeString>()}, - {"disk", std::make_shared<DataTypeString>()}, - {"reason", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())}, - {"min_block_number", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt64>())}, - {"max_block_number", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt64>())}, - {"level", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt32>())} - }}); - setInMemoryMetadata(storage_metadata); - } - - Pipes read( - const Names & /* column_names */, - const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum /*processed_stage*/, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) override - { - StoragesInfoStream stream(query_info, context); - - /// Create the result. - Block block = metadata_snapshot->getSampleBlock(); - MutableColumns new_columns = block.cloneEmptyColumns(); - - while (StoragesInfo info = stream.next()) + const auto parts = info.data->getDetachedParts(); + for (const auto & p : parts) { - const auto parts = info.data->getDetachedParts(); - for (const auto & p : parts) - { - size_t i = 0; - new_columns[i++]->insert(info.database); - new_columns[i++]->insert(info.table); - new_columns[i++]->insert(p.valid_name ? p.partition_id : Field()); - new_columns[i++]->insert(p.dir_name); - new_columns[i++]->insert(p.disk); - new_columns[i++]->insert(p.valid_name ? p.prefix : Field()); - new_columns[i++]->insert(p.valid_name ? p.min_block : Field()); - new_columns[i++]->insert(p.valid_name ? p.max_block : Field()); - new_columns[i++]->insert(p.valid_name ? p.level : Field()); - } + size_t i = 0; + new_columns[i++]->insert(info.database); + new_columns[i++]->insert(info.table); + new_columns[i++]->insert(p.valid_name ? p.partition_id : Field()); + new_columns[i++]->insert(p.dir_name); + new_columns[i++]->insert(p.disk); + new_columns[i++]->insert(p.valid_name ? p.prefix : Field()); + new_columns[i++]->insert(p.valid_name ? p.min_block : Field()); + new_columns[i++]->insert(p.valid_name ? p.max_block : Field()); + new_columns[i++]->insert(p.valid_name ? p.level : Field()); } - - UInt64 num_rows = new_columns.at(0)->size(); - Chunk chunk(std::move(new_columns), num_rows); - - Pipes pipes; - pipes.emplace_back(std::make_shared<SourceFromSingleChunk>(std::move(block), std::move(chunk))); - return pipes; } -}; -StoragePtr -createDetachedPartsTable() -{ - return StorageSystemDetachedParts::create(StorageID{"system", "detached_parts"}); + UInt64 num_rows = new_columns.at(0)->size(); + Chunk chunk(std::move(new_columns), num_rows); + + Pipes pipes; + pipes.emplace_back(std::make_shared<SourceFromSingleChunk>(std::move(block), std::move(chunk))); + return pipes; } } diff --git a/src/Storages/System/StorageSystemDetachedParts.h b/src/Storages/System/StorageSystemDetachedParts.h index 3fea29e5b42..9db6d687ba8 100644 --- a/src/Storages/System/StorageSystemDetachedParts.h +++ b/src/Storages/System/StorageSystemDetachedParts.h @@ -1,10 +1,35 @@ #pragma once -#include <Storages/IStorage_fwd.h> +#include <Storages/IStorage.h> namespace DB { -StoragePtr createDetachedPartsTable(); +/** + * Implements system table 'detached_parts' which allows to get information + * about detached data parts for tables of MergeTree family. + * We don't use StorageSystemPartsBase, because it introduces virtual _state + * column and column aliases which we don't need. + */ +class StorageSystemDetachedParts final : + public ext::shared_ptr_helper<StorageSystemDetachedParts>, + public IStorage +{ + friend struct ext::shared_ptr_helper<StorageSystemDetachedParts>; +public: + std::string getName() const override { return "SystemDetachedParts"; } + +protected: + explicit StorageSystemDetachedParts(const StorageID & table_id_); + + Pipes read( + const Names & /* column_names */, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum /*processed_stage*/, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) override; +}; } diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 89c8de7f430..5aed801cd05 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -69,19 +69,23 @@ namespace DB namespace { -template<typename StorageT> -void attach(IDatabase & system_database, const String & table_name) +template<typename StorageT, typename... StorageArgs> +void attach(IDatabase & system_database, const String & table_name, StorageArgs && ... args) { if (system_database.getUUID() == UUIDHelpers::Nil) { + /// Attach to Ordinary database auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name); - system_database.attachTable(table_name, StorageT::create(table_id)); + system_database.attachTable(table_name, StorageT::create(table_id, std::forward<StorageArgs>(args)...)); } else { + /// Attach to Atomic database + /// NOTE: UUIDs are not persistent, but it's ok since no data are stored on disk for these storages + /// and path is actually not used auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name, UUIDHelpers::generateV4()); String path = "store/" + DatabaseCatalog::getPathForUUID(table_id.uuid); - system_database.attachTable(table_name, StorageT::create(table_id), path); + system_database.attachTable(table_name, StorageT::create(table_id, std::forward<StorageArgs>(args)...), path); } } @@ -92,12 +96,10 @@ void attach(IDatabase & system_database, const String & table_name) void attachSystemTablesLocal(IDatabase & system_database) { attach<StorageSystemOne>(system_database, "one"); - - //system_database.attachTable("numbers", StorageSystemNumbers::create(StorageID("system", "numbers"), false)); - //system_database.attachTable("numbers_mt", StorageSystemNumbers::create(StorageID("system", "numbers_mt"), true)); - //system_database.attachTable("zeros", StorageSystemZeros::create(StorageID("system", "zeros"), false)); - //system_database.attachTable("zeros_mt", StorageSystemZeros::create(StorageID("system", "zeros_mt"), true)); - + attach<StorageSystemNumbers>(system_database, "numbers", false); + attach<StorageSystemNumbers>(system_database, "numbers_mt", true); + attach<StorageSystemZeros>(system_database, "zeros", false); + attach<StorageSystemZeros>(system_database, "zeros_mt", true); attach<StorageSystemDatabases>(system_database, "databases"); attach<StorageSystemTables>(system_database, "tables"); attach<StorageSystemColumns>(system_database, "columns"); @@ -141,7 +143,7 @@ void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper) attachSystemTablesLocal(system_database); attach<StorageSystemParts>(system_database, "parts"); - //attach<>(system_database, "detached_parts", createDetachedPartsTable()); + attach<StorageSystemDetachedParts>(system_database, "detached_parts"); attach<StorageSystemPartsColumns>(system_database, "parts_columns"); attach<StorageSystemDisks>(system_database, "disks"); attach<StorageSystemStoragePolicies>(system_database, "storage_policies"); @@ -162,9 +164,9 @@ void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper) attach<StorageSystemZooKeeper>(system_database, "zookeeper"); } -void attachSystemTablesAsync(IDatabase & /*system_database*/, AsynchronousMetrics & /*async_metrics*/) +void attachSystemTablesAsync(IDatabase & system_database, AsynchronousMetrics & async_metrics) { - //system_database.attachTable("asynchronous_metrics", StorageSystemAsynchronousMetrics::create("asynchronous_metrics", async_metrics)); + attach<StorageSystemAsynchronousMetrics>(system_database, "asynchronous_metrics", async_metrics); } } From a5d74c44f10c6ded614e8cf2259ba81255677f66 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov <avtokmakov@yandex-team.ru> Date: Thu, 13 Aug 2020 20:55:12 +0300 Subject: [PATCH 306/374] run stress tests with Atomic database --- docker/test/stress/stress | 22 ++++++++++++++-------- tests/clickhouse-test | 11 ++++++++--- 2 files changed, 22 insertions(+), 11 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index b107dc59829..173cb8442d6 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -24,20 +24,26 @@ def run_perf_test(cmd, xmls_path, output_folder): return p +def get_options(i): + options = "" + if 0 < i: + options += " --order=random" + if i == 1: + options += " --atomic-db-engine" + return options + + def run_func_test(cmd, output_prefix, num_processes, skip_tests_option): skip_list_opt = get_skip_list_cmd(cmd) output_paths = [os.path.join(output_prefix, "stress_test_run_{}.txt".format(i)) for i in range(num_processes)] - f = open(output_paths[0], 'w') - main_command = "{} {} {}".format(cmd, skip_list_opt, skip_tests_option) - logging.info("Run func tests main cmd '%s'", main_command) - pipes = [Popen(main_command, shell=True, stdout=f, stderr=f)] - for output_path in output_paths[1:]: - time.sleep(0.5) - f = open(output_path, 'w') - full_command = "{} {} --order=random {}".format(cmd, skip_list_opt, skip_tests_option) + pipes = [] + for i in range(0, len(output_paths)): + f = open(output_paths[i], 'w') + full_command = "{} {} {} {}".format(cmd, skip_list_opt, get_options(i), skip_tests_option) logging.info("Run func tests '%s'", full_command) p = Popen(full_command, shell=True, stdout=f, stderr=f) pipes.append(p) + time.sleep(0.5) return pipes diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 12629628007..857418193d7 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -49,6 +49,10 @@ def remove_control_characters(s): s = re.sub(r"[\x00-\x08\x0b\x0e-\x1f\x7f]", "", s) return s +def get_db_engine(args): + if args.atomic_db_engine: + return " ENGINE=Atomic" + return "" def run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file): @@ -69,7 +73,7 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std database = 'test_{suffix}'.format(suffix=random_str()) clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - clickhouse_proc_create.communicate("CREATE DATABASE " + database) + clickhouse_proc_create.communicate("CREATE DATABASE " + database + get_db_engine(args)) os.environ["CLICKHOUSE_DATABASE"] = database @@ -507,10 +511,10 @@ def main(args): if args.database and args.database != "test": clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS " + args.database) + clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS " + args.database + get_db_engine(args)) clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS test") + clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS test" + get_db_engine(args)) def is_test_from_dir(suite_dir, case): case_file = os.path.join(suite_dir, case) @@ -722,6 +726,7 @@ if __name__ == '__main__': parser.add_argument('-r', '--server-check-retries', default=30, type=int, help='Num of tries to execute SELECT 1 before tests started') parser.add_argument('--skip-list-path', help="Path to skip-list file") parser.add_argument('--use-skip-list', action='store_true', default=False, help="Use skip list to skip tests if found") + parser.add_argument('--atomic-db-engine', action='store_true', help='Create databases with Atomic engine by default') parser.add_argument('--no-stateless', action='store_true', help='Disable all stateless tests') parser.add_argument('--no-stateful', action='store_true', help='Disable all stateful tests') From cd3155ff199d14f07235a6e0ce3523f3209e07e4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov <avtokmakov@yandex-team.ru> Date: Thu, 13 Aug 2020 21:05:31 +0300 Subject: [PATCH 307/374] fix --- src/Interpreters/loadMetadata.cpp | 3 ++- src/Storages/System/attachSystemTables.cpp | 1 - 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index e273d6a7be1..bb476999f8c 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -170,7 +170,8 @@ void loadMetadata(Context & context, const String & default_database_name) void loadMetadataSystem(Context & context) { String path = context.getPath() + "metadata/" + DatabaseCatalog::SYSTEM_DATABASE; - if (Poco::File(path).exists()) + String metadata_file = path + ".sql"; + if (Poco::File(path).exists() || Poco::File(metadata_file).exists()) { /// 'has_force_restore_data_flag' is true, to not fail on loading query_log table, if it is corrupted. loadDatabase(context, DatabaseCatalog::SYSTEM_DATABASE, path, true); diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 5aed801cd05..15edd096b2f 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -92,7 +92,6 @@ void attach(IDatabase & system_database, const String & table_name, StorageArgs } -///TODO allow store system tables in DatabaseAtomic void attachSystemTablesLocal(IDatabase & system_database) { attach<StorageSystemOne>(system_database, "one"); From 17b27b6c286cf84b8841251e28455872d021d486 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin <a3at.mail@gmail.com> Date: Thu, 13 Aug 2020 22:03:11 +0300 Subject: [PATCH 308/374] Do not optimize any(arrayJoin()) -> arrayJoin() under optimize_move_functions_out_of_any MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Otherwise the following query will be optimized incorrectly: SELECT *, any(arrayJoin([[], []])) FROM numbers(1) GROUP BY number And the result will be: ┌─number─┬─arrayJoin(array(array(), array()))─┐ │ 0 │ [] │ │ 0 │ [] │ └────────┴────────────────────────────────────┘ While should be: ┌─number─┬─any(arrayJoin(array(array(), array())))─┐ │ 0 │ [] │ └────────┴─────────────────────────────────────────┘ --- src/Interpreters/RewriteAnyFunctionVisitor.cpp | 15 +++++++++++++++ .../01322_any_input_optimize.reference | 2 ++ .../0_stateless/01322_any_input_optimize.sql | 3 +++ 3 files changed, 20 insertions(+) diff --git a/src/Interpreters/RewriteAnyFunctionVisitor.cpp b/src/Interpreters/RewriteAnyFunctionVisitor.cpp index 3652f49cdda..cb6debfc731 100644 --- a/src/Interpreters/RewriteAnyFunctionVisitor.cpp +++ b/src/Interpreters/RewriteAnyFunctionVisitor.cpp @@ -20,6 +20,21 @@ bool extractIdentifiers(const ASTFunction & func, std::unordered_set<ASTPtr *> & { if (const auto * arg_func = arg->as<ASTFunction>()) { + /// arrayJoin() is special and should not be optimized (think about + /// it as a an aggregate function), otherwise wrong result will be + /// produced: + /// SELECT *, any(arrayJoin([[], []])) FROM numbers(1) GROUP BY number + /// ┌─number─┬─arrayJoin(array(array(), array()))─┐ + /// │ 0 │ [] │ + /// │ 0 │ [] │ + /// └────────┴────────────────────────────────────┘ + /// While should be: + /// ┌─number─┬─any(arrayJoin(array(array(), array())))─┐ + /// │ 0 │ [] │ + /// └────────┴─────────────────────────────────────────┘ + if (arg_func->name == "arrayJoin") + return false; + if (arg_func->name == "lambda") return false; diff --git a/tests/queries/0_stateless/01322_any_input_optimize.reference b/tests/queries/0_stateless/01322_any_input_optimize.reference index 4b6fbfd32c0..c02c9fbeae4 100644 --- a/tests/queries/0_stateless/01322_any_input_optimize.reference +++ b/tests/queries/0_stateless/01322_any_input_optimize.reference @@ -28,3 +28,5 @@ SELECT x FROM numbers(1, 2) 6 6 +arrayJoin +0 [] diff --git a/tests/queries/0_stateless/01322_any_input_optimize.sql b/tests/queries/0_stateless/01322_any_input_optimize.sql index 4b8a55d4c7b..c868bad659e 100644 --- a/tests/queries/0_stateless/01322_any_input_optimize.sql +++ b/tests/queries/0_stateless/01322_any_input_optimize.sql @@ -30,3 +30,6 @@ ANALYZE SELECT anyLast(number * 3) AS x, x FROM numbers(1, 2); SELECT anyLast(number * 3) AS x, x FROM numbers(1, 2); SELECT any(anyLast(number)) FROM numbers(1); -- { serverError 184 } + +SELECT 'arrayJoin'; +SELECT *, any(arrayJoin([[], []])) FROM numbers(1) GROUP BY number; From d1ef34adff175f369518b3794c8c165c86053cd8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov <avtokmakov@yandex-team.ru> Date: Thu, 13 Aug 2020 22:41:06 +0300 Subject: [PATCH 309/374] fix --- programs/local/LocalServer.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index ca348382281..5cdf5766a44 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -247,12 +247,15 @@ try context->setCurrentDatabase(default_database); applyCmdOptions(); - if (!context->getPath().empty()) + String path = context->getPath(); + if (!path.empty()) { /// Lock path directory before read status.emplace(context->getPath() + "status", StatusFile::write_full_info); - LOG_DEBUG(log, "Loading metadata from {}", context->getPath()); + LOG_DEBUG(log, "Loading metadata from {}", path); + Poco::File(path + "data/").createDirectories(); + Poco::File(path + "metadata/").createDirectories(); loadMetadataSystem(*context); attachSystemTables(*context); loadMetadata(*context); From 89419ceb9bd60121a029430b26aa4506c517f01d Mon Sep 17 00:00:00 2001 From: alexey-milovidov <milovidov@yandex-team.ru> Date: Thu, 13 Aug 2020 23:10:06 +0300 Subject: [PATCH 310/374] Revert "cmake: Add option to fail configuration instead of auto-reconfiguration" --- CMakeLists.txt | 14 +- cmake/analysis.cmake | 4 +- cmake/contrib_finder.cmake | 4 +- cmake/find/amqpcpp.cmake | 25 +- cmake/find/avro.cmake | 18 +- cmake/find/base64.cmake | 26 +- cmake/find/brotli.cmake | 14 +- cmake/find/capnp.cmake | 26 +- cmake/find/cassandra.cmake | 40 +- cmake/find/ccache.cmake | 10 +- cmake/find/cxx.cmake | 71 +-- cmake/find/fastops.cmake | 27 +- cmake/find/gperf.cmake | 11 +- cmake/find/grpc.cmake | 50 +- cmake/find/gtest.cmake | 17 +- cmake/find/h3.cmake | 38 +- cmake/find/hdfs3.cmake | 22 +- cmake/find/icu.cmake | 13 +- cmake/find/ldap.cmake | 155 +++--- cmake/find/libgsasl.cmake | 17 +- cmake/find/libxml2.cmake | 5 - cmake/find/llvm.cmake | 133 +++--- cmake/find/ltdl.cmake | 2 +- cmake/find/msgpack.cmake | 30 +- cmake/find/mysqlclient.cmake | 93 ++-- cmake/find/opencl.cmake | 10 +- cmake/find/orc.cmake | 19 +- cmake/find/parquet.cmake | 40 +- cmake/find/protobuf.cmake | 13 +- cmake/find/rapidjson.cmake | 7 - cmake/find/rdkafka.cmake | 55 +-- cmake/find/re2.cmake | 7 +- cmake/find/s3.cmake | 47 +- cmake/find/sentry.cmake | 5 - cmake/find/simdjson.cmake | 7 +- cmake/find/snappy.cmake | 26 +- cmake/find/sparsehash.cmake | 3 - cmake/find/ssl.cmake | 15 +- cmake/find/stats.cmake | 4 - cmake/find/termcap.cmake | 3 - cmake/find/zlib.cmake | 3 - cmake/find/zstd.cmake | 6 +- cmake/tools.cmake | 22 +- contrib/arrow-cmake/CMakeLists.txt | 2 +- contrib/boost-cmake/CMakeLists.txt | 82 ++-- contrib/cctz-cmake/CMakeLists.txt | 64 ++- contrib/curl-cmake/CMakeLists.txt | 363 +++++++------- contrib/hyperscan-cmake/CMakeLists.txt | 494 ++++++++++--------- contrib/jemalloc-cmake/CMakeLists.txt | 293 ++++++------ contrib/libcpuid-cmake/CMakeLists.txt | 55 +-- contrib/libhdfs3-cmake/CMakeLists.txt | 12 +- contrib/lz4-cmake/CMakeLists.txt | 37 +- contrib/poco-cmake/CMakeLists.txt | 3 - contrib/replxx-cmake/CMakeLists.txt | 88 ++-- contrib/unixodbc-cmake/CMakeLists.txt | 630 ++++++++++++------------- 55 files changed, 1411 insertions(+), 1869 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index ec2cc3cfdf8..c43b881d482 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -28,14 +28,6 @@ endforeach() project(ClickHouse) -option(FAIL_ON_UNSUPPORTED_OPTIONS_COMBINATION - "Stop/Fail CMake configuration if some ENABLE_XXX option is defined (either ON or OFF) but is not possible to satisfy" OFF) -if(FAIL_ON_UNSUPPORTED_OPTIONS_COMBINATION) - set(RECONFIGURE_MESSAGE_LEVEL FATAL_ERROR) -else() - set(RECONFIGURE_MESSAGE_LEVEL STATUS) -endif() - include (cmake/arch.cmake) include (cmake/target.cmake) include (cmake/tools.cmake) @@ -65,7 +57,7 @@ if(ENABLE_IPO) message(STATUS "IPO/LTO is supported, enabling") set(CMAKE_INTERPROCEDURAL_OPTIMIZATION TRUE) else() - message (${RECONFIGURE_MESSAGE_LEVEL} "IPO/LTO is not supported: <${IPO_NOT_SUPPORTED}>") + message(STATUS "IPO/LTO is not supported: <${IPO_NOT_SUPPORTED}>") endif() else() message(STATUS "IPO/LTO not enabled.") @@ -141,8 +133,6 @@ option (ENABLE_TESTS "Enables tests" ON) if (OS_LINUX AND NOT UNBUNDLED AND MAKE_STATIC_LIBRARIES AND NOT SPLIT_SHARED_LIBRARIES AND CMAKE_VERSION VERSION_GREATER "3.9.0") option (GLIBC_COMPATIBILITY "Set to TRUE to enable compatibility with older glibc libraries. Only for x86_64, Linux. Implies ENABLE_FASTMEMCPY." ON) -elseif(GLIBC_COMPATIBILITY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Glibc compatibility cannot be enabled in current configuration") endif () if (NOT CMAKE_VERSION VERSION_GREATER "3.9.0") @@ -266,8 +256,6 @@ if (COMPILER_CLANG) set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -flto=thin") set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -flto=thin") set (CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO "${CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO} -flto=thin") - elseif (ENABLE_THINLTO) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot enable ThinLTO") endif () # Always prefer llvm tools when using clang. For instance, we cannot use GNU ar when llvm LTO is enabled diff --git a/cmake/analysis.cmake b/cmake/analysis.cmake index daaa730ac4b..287c36a8de7 100644 --- a/cmake/analysis.cmake +++ b/cmake/analysis.cmake @@ -12,9 +12,7 @@ if (ENABLE_CLANG_TIDY) set (USE_CLANG_TIDY 1) # The variable CMAKE_CXX_CLANG_TIDY will be set inside src and base directories with non third-party code. # set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") - elseif (FAIL_ON_UNSUPPORTED_OPTIONS_COMBINATION) - message(FATAL_ERROR "clang-tidy is not found") else () - message(STATUS "clang-tidy is not found. This is normal - the tool is only used for static code analysis and isn't essential for the build.") + message(STATUS "clang-tidy is not found. This is normal - the tool is used only for static code analysis and not essential for build.") endif () endif () diff --git a/cmake/contrib_finder.cmake b/cmake/contrib_finder.cmake index 64c6d5f5c0a..f9bf3ce2837 100644 --- a/cmake/contrib_finder.cmake +++ b/cmake/contrib_finder.cmake @@ -8,9 +8,6 @@ macro(find_contrib_lib LIB_NAME) if (NOT USE_INTERNAL_${LIB_NAME_UC}_LIBRARY) find_package ("${LIB_NAME}") - if (NOT ${LIB_NAME_UC}_FOUND) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use system ${LIB_NAME}") - endif() endif () if (NOT ${LIB_NAME_UC}_FOUND) @@ -20,4 +17,5 @@ macro(find_contrib_lib LIB_NAME) endif () message (STATUS "Using ${LIB_NAME}: ${${LIB_NAME_UC}_INCLUDE_DIR} : ${${LIB_NAME_UC}_LIBRARIES}") + endmacro() diff --git a/cmake/find/amqpcpp.cmake b/cmake/find/amqpcpp.cmake index 4191dce26bb..b3e193c72ff 100644 --- a/cmake/find/amqpcpp.cmake +++ b/cmake/find/amqpcpp.cmake @@ -1,22 +1,21 @@ option(ENABLE_AMQPCPP "Enalbe AMQP-CPP" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_AMQPCPP) - return() -endif() - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP/CMakeLists.txt") message (WARNING "submodule contrib/AMQP-CPP is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal AMQP-CPP library") - set (USE_AMQPCPP 0) - return() + set (ENABLE_AMQPCPP 0) endif () -set (USE_AMQPCPP 1) -set (AMQPCPP_LIBRARY AMQP-CPP) +if (ENABLE_AMQPCPP) -set (AMQPCPP_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP/include") -list (APPEND AMQPCPP_INCLUDE_DIR - "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP/include" - "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP") + set (USE_AMQPCPP 1) + set (AMQPCPP_LIBRARY AMQP-CPP) + + set (AMQPCPP_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP/include") + + list (APPEND AMQPCPP_INCLUDE_DIR + "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP/include" + "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP") + +endif() message (STATUS "Using AMQP-CPP=${USE_AMQPCPP}: ${AMQPCPP_INCLUDE_DIR} : ${AMQPCPP_LIBRARY}") diff --git a/cmake/find/avro.cmake b/cmake/find/avro.cmake index a07e6353212..cdb3fc84d3d 100644 --- a/cmake/find/avro.cmake +++ b/cmake/find/avro.cmake @@ -1,34 +1,28 @@ option (ENABLE_AVRO "Enable Avro" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_AVRO) - if (USE_INTERNAL_AVRO_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal avro library with ENABLE_AVRO=OFF") - endif() - return() -endif() +if (ENABLE_AVRO) option (USE_INTERNAL_AVRO_LIBRARY "Set to FALSE to use system avro library instead of bundled" ${NOT_UNBUNDLED}) -if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/avro/lang/c++/CMakeLists.txt") - if (USE_INTERNAL_AVRO_LIBRARY) +if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/avro/lang/c++/CMakeLists.txt") + if(USE_INTERNAL_AVRO_LIBRARY) message(WARNING "submodule contrib/avro is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot find internal avro") - set(USE_INTERNAL_AVRO_LIBRARY 0) endif() set(MISSING_INTERNAL_AVRO_LIBRARY 1) + set(USE_INTERNAL_AVRO_LIBRARY 0) endif() if (NOT USE_INTERNAL_AVRO_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Using system avro library is not supported yet") elseif(NOT MISSING_INTERNAL_AVRO_LIBRARY) include(cmake/find/snappy.cmake) set(AVROCPP_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/avro/lang/c++/include") set(AVROCPP_LIBRARY avrocpp) - set(USE_INTERNAL_AVRO_LIBRARY 1) endif () if (AVROCPP_LIBRARY AND AVROCPP_INCLUDE_DIR) set(USE_AVRO 1) endif() +endif() + message (STATUS "Using avro=${USE_AVRO}: ${AVROCPP_INCLUDE_DIR} : ${AVROCPP_LIBRARY}") diff --git a/cmake/find/base64.cmake b/cmake/find/base64.cmake index 7427baf9cad..f72397597d7 100644 --- a/cmake/find/base64.cmake +++ b/cmake/find/base64.cmake @@ -1,21 +1,17 @@ -option (ENABLE_BASE64 "Enable base64" ${ENABLE_LIBRARIES}) - -if (NOT ENABLE_BASE64) - return() -endif() - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/base64/LICENSE") set (MISSING_INTERNAL_BASE64_LIBRARY 1) message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init --recursive") endif () -if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/base64") - message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init --recursive") -else() - set (BASE64_LIBRARY base64) - set (USE_BASE64 1) -endif() +if (NOT MISSING_INTERNAL_BASE64_LIBRARY) + option (ENABLE_BASE64 "Enable base64" ${ENABLE_LIBRARIES}) +endif () -if (NOT USE_BASE64) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot enable base64") -endif() +if (ENABLE_BASE64) + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/base64") + message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init --recursive") + else() + set (BASE64_LIBRARY base64) + set (USE_BASE64 1) + endif() +endif () diff --git a/cmake/find/brotli.cmake b/cmake/find/brotli.cmake index eca3df0d2c8..5453cf15236 100644 --- a/cmake/find/brotli.cmake +++ b/cmake/find/brotli.cmake @@ -1,18 +1,12 @@ option (ENABLE_BROTLI "Enable brotli" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_BROTLI) - if (USE_INTERNAL_BROTLI_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal brotly library with ENABLE_BROTLI=OFF") - endif() - return() -endif() +if (ENABLE_BROTLI) option (USE_INTERNAL_BROTLI_LIBRARY "Set to FALSE to use system libbrotli library instead of bundled" ${NOT_UNBUNDLED}) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/brotli/c/include/brotli/decode.h") if (USE_INTERNAL_BROTLI_LIBRARY) message (WARNING "submodule contrib/brotli is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot find internal brotli") set (USE_INTERNAL_BROTLI_LIBRARY 0) endif () set (MISSING_INTERNAL_BROTLI_LIBRARY 1) @@ -25,18 +19,18 @@ if(NOT USE_INTERNAL_BROTLI_LIBRARY) find_path(BROTLI_INCLUDE_DIR NAMES brotli/decode.h brotli/encode.h brotli/port.h brotli/types.h PATHS ${BROTLI_INCLUDE_PATHS}) if(BROTLI_LIBRARY_DEC AND BROTLI_LIBRARY_ENC AND BROTLI_LIBRARY_COMMON) set(BROTLI_LIBRARY ${BROTLI_LIBRARY_DEC} ${BROTLI_LIBRARY_ENC} ${BROTLI_LIBRARY_COMMON}) - else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use system brotli") endif() endif() if (BROTLI_LIBRARY AND BROTLI_INCLUDE_DIR) set (USE_BROTLI 1) elseif (NOT MISSING_INTERNAL_BROTLI_LIBRARY) - set (BROTLI_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/brotli/c/include") + set (BROTLI_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/brotli/c/include) set (USE_INTERNAL_BROTLI_LIBRARY 1) set (BROTLI_LIBRARY brotli) set (USE_BROTLI 1) endif () +endif() + message (STATUS "Using brotli=${USE_BROTLI}: ${BROTLI_INCLUDE_DIR} : ${BROTLI_LIBRARY}") diff --git a/cmake/find/capnp.cmake b/cmake/find/capnp.cmake index ee4735bd175..0620a66808b 100644 --- a/cmake/find/capnp.cmake +++ b/cmake/find/capnp.cmake @@ -1,21 +1,15 @@ option (ENABLE_CAPNP "Enable Cap'n Proto" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_CAPNP) - if (USE_INTERNAL_CAPNP_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal capnproto library with ENABLE_CAPNP=OFF") - endif() - return() -endif() +if (ENABLE_CAPNP) option (USE_INTERNAL_CAPNP_LIBRARY "Set to FALSE to use system capnproto library instead of bundled" ${NOT_UNBUNDLED}) if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/capnproto/CMakeLists.txt") if(USE_INTERNAL_CAPNP_LIBRARY) message(WARNING "submodule contrib/capnproto is missing. to fix try run: \n git submodule update --init --recursive") - message(${RECONFIGURE_MESSAGE_LEVEL} "cannot find internal capnproto") - set(USE_INTERNAL_CAPNP_LIBRARY 0) endif() set(MISSING_INTERNAL_CAPNP_LIBRARY 1) + set(USE_INTERNAL_CAPNP_LIBRARY 0) endif() # FIXME: refactor to use `add_library(… IMPORTED)` if possible. @@ -24,21 +18,17 @@ if (NOT USE_INTERNAL_CAPNP_LIBRARY) find_library (CAPNP capnp) find_library (CAPNPC capnpc) - if(KJ AND CAPNP AND CAPNPC) - set (CAPNP_LIBRARIES ${CAPNPC} ${CAPNP} ${KJ}) - else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system capnproto") - endif() -endif() - -if (CAPNP_LIBRARIES) - set (USE_CAPNP 1) + set (CAPNP_LIBRARIES ${CAPNPC} ${CAPNP} ${KJ}) elseif(NOT MISSING_INTERNAL_CAPNP_LIBRARY) add_subdirectory(contrib/capnproto-cmake) set (CAPNP_LIBRARIES capnpc) +endif () + +if (CAPNP_LIBRARIES) set (USE_CAPNP 1) - set (USE_INTERNAL_CAPNP_LIBRARY 1) +endif () + endif () message (STATUS "Using capnp=${USE_CAPNP}: ${CAPNP_LIBRARIES}") diff --git a/cmake/find/cassandra.cmake b/cmake/find/cassandra.cmake index e28f1534c09..f41e0f645f4 100644 --- a/cmake/find/cassandra.cmake +++ b/cmake/find/cassandra.cmake @@ -1,29 +1,25 @@ option(ENABLE_CASSANDRA "Enable Cassandra" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_CASSANDRA) - return() -endif() - -if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libuv") - message (ERROR "submodule contrib/libuv is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libuv needed for Cassandra") -elseif (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cassandra") - message (ERROR "submodule contrib/cassandra is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal Cassandra") -else() - set (LIBUV_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/libuv") - set (CASSANDRA_INCLUDE_DIR - "${ClickHouse_SOURCE_DIR}/contrib/cassandra/include/") - if (MAKE_STATIC_LIBRARIES) - set (LIBUV_LIBRARY uv_a) - set (CASSANDRA_LIBRARY cassandra_static) +if (ENABLE_CASSANDRA) + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libuv") + message (ERROR "submodule contrib/libuv is missing. to fix try run: \n git submodule update --init --recursive") + elseif (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cassandra") + message (ERROR "submodule contrib/cassandra is missing. to fix try run: \n git submodule update --init --recursive") else() - set (LIBUV_LIBRARY uv) - set (CASSANDRA_LIBRARY cassandra) - endif() + set (LIBUV_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/libuv") + set (CASSANDRA_INCLUDE_DIR + "${ClickHouse_SOURCE_DIR}/contrib/cassandra/include/") + if (USE_STATIC_LIBRARIES) + set (LIBUV_LIBRARY uv_a) + set (CASSANDRA_LIBRARY cassandra_static) + else() + set (LIBUV_LIBRARY uv) + set (CASSANDRA_LIBRARY cassandra) + endif() + set (USE_CASSANDRA 1) + set (CASS_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/cassandra") - set (USE_CASSANDRA 1) - set (CASS_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/cassandra") + endif() endif() message (STATUS "Using cassandra=${USE_CASSANDRA}: ${CASSANDRA_INCLUDE_DIR} : ${CASSANDRA_LIBRARY}") diff --git a/cmake/find/ccache.cmake b/cmake/find/ccache.cmake index e20e6416e97..95d6b208cfa 100644 --- a/cmake/find/ccache.cmake +++ b/cmake/find/ccache.cmake @@ -1,9 +1,3 @@ -option(ENABLE_CCACHE "Speedup re-compilations using ccache" ON) - -if (NOT ENABLE_CCACHE) - return() -endif() - find_program (CCACHE_FOUND ccache) if (CCACHE_FOUND AND NOT CMAKE_CXX_COMPILER_LAUNCHER MATCHES "ccache" AND NOT CMAKE_CXX_COMPILER MATCHES "ccache") @@ -15,8 +9,6 @@ if (CCACHE_FOUND AND NOT CMAKE_CXX_COMPILER_LAUNCHER MATCHES "ccache" AND NOT CM set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE ${CCACHE_FOUND}) set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK ${CCACHE_FOUND}) else () - message(${RECONFIGURE_MESSAGE_LEVEL} "Not using ${CCACHE_FOUND} ${CCACHE_VERSION} bug: https://bugzilla.samba.org/show_bug.cgi?id=8118") + message(STATUS "Not using ${CCACHE_FOUND} ${CCACHE_VERSION} bug: https://bugzilla.samba.org/show_bug.cgi?id=8118") endif () -elseif (NOT CCACHE_FOUND) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use ccache") endif () diff --git a/cmake/find/cxx.cmake b/cmake/find/cxx.cmake index 02f7113e6fb..f6c999351e8 100644 --- a/cmake/find/cxx.cmake +++ b/cmake/find/cxx.cmake @@ -1,68 +1,41 @@ -option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++" ${NOT_UNBUNDLED}) - -if (NOT USE_LIBCXX) - if (USE_INTERNAL_LIBCXX_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal libcxx with USE_LIBCXX=OFF") - endif() - target_link_libraries(global-libs INTERFACE -l:libstdc++.a -l:libstdc++fs.a) # Always link these libraries as static - target_link_libraries(global-libs INTERFACE ${EXCEPTION_HANDLING_LIBRARY}) - return() -endif() - set(USE_INTERNAL_LIBCXX_LIBRARY_DEFAULT ${NOT_UNBUNDLED}) -option (USE_INTERNAL_LIBCXX_LIBRARY "Set to FALSE to use system libcxx and libcxxabi libraries instead of bundled" ${USE_INTERNAL_LIBCXX_LIBRARY_DEFAULT}) if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libcxx/CMakeLists.txt") - if (USE_INTERNAL_LIBCXX_LIBRARY) - message(WARNING "submodule contrib/libcxx is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libcxx") - set(USE_INTERNAL_LIBCXX_LIBRARY 0) - endif() + message(WARNING "submodule contrib/libcxx is missing. to fix try run: \n git submodule update --init --recursive") set(USE_INTERNAL_LIBCXX_LIBRARY_DEFAULT 0) - set(MISSING_INTERNAL_LIBCXX_LIBRARY 1) endif() -set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=0") # More checks in debug build. +option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++" ${NOT_UNBUNDLED}) +option (USE_INTERNAL_LIBCXX_LIBRARY "Set to FALSE to use system libcxx and libcxxabi libraries instead of bundled" ${USE_INTERNAL_LIBCXX_LIBRARY_DEFAULT}) -if (NOT USE_INTERNAL_LIBCXX_LIBRARY) - find_library (LIBCXX_LIBRARY c++) - find_library (LIBCXXFS_LIBRARY c++fs) - find_library (LIBCXXABI_LIBRARY c++abi) +if (USE_LIBCXX) + set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=0") # More checks in debug build. - if(LIBCXX_LIBRARY AND LIBCXXABI_LIBRARY) # c++fs is now a part of the libc++ - set (HAVE_LIBCXX 1) + if (NOT USE_INTERNAL_LIBCXX_LIBRARY) + find_library (LIBCXX_LIBRARY c++) + find_library (LIBCXXFS_LIBRARY c++fs) + find_library (LIBCXXABI_LIBRARY c++abi) + + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") + + target_link_libraries(global-libs INTERFACE ${EXCEPTION_HANDLING_LIBRARY}) else () - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system libcxx") - endif() + set (LIBCXX_LIBRARY cxx) + set (LIBCXXABI_LIBRARY cxxabi) + add_subdirectory(contrib/libcxxabi-cmake) + add_subdirectory(contrib/libcxx-cmake) - if(NOT LIBCXXFS_LIBRARY) - set(LIBCXXFS_LIBRARY ${LIBCXX_LIBRARY}) - endif() + # Exception handling library is embedded into libcxxabi. + endif () - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") - - target_link_libraries(global-libs INTERFACE ${EXCEPTION_HANDLING_LIBRARY}) -endif () - -if (NOT HAVE_LIBCXX AND NOT MISSING_INTERNAL_LIBCXX_LIBRARY) - set (LIBCXX_LIBRARY cxx) - set (LIBCXXABI_LIBRARY cxxabi) - add_subdirectory(contrib/libcxxabi-cmake) - add_subdirectory(contrib/libcxx-cmake) - - # Exception handling library is embedded into libcxxabi. + target_link_libraries(global-libs INTERFACE ${LIBCXX_LIBRARY} ${LIBCXXABI_LIBRARY} ${LIBCXXFS_LIBRARY}) set (HAVE_LIBCXX 1) - set(USE_INTERNAL_LIBCXX_LIBRARY 1) -endif () - -if (HAVE_LIBCXX) - target_link_libraries(global-libs INTERFACE ${LIBCXX_LIBRARY} ${LIBCXXABI_LIBRARY} ${LIBCXXFS_LIBRARY}) message (STATUS "Using libcxx: ${LIBCXX_LIBRARY}") message (STATUS "Using libcxxfs: ${LIBCXXFS_LIBRARY}") message (STATUS "Using libcxxabi: ${LIBCXXABI_LIBRARY}") -else() +else () target_link_libraries(global-libs INTERFACE -l:libstdc++.a -l:libstdc++fs.a) # Always link these libraries as static target_link_libraries(global-libs INTERFACE ${EXCEPTION_HANDLING_LIBRARY}) -endif() +endif () diff --git a/cmake/find/fastops.cmake b/cmake/find/fastops.cmake index 5ab320bdb7a..1296ba586a0 100644 --- a/cmake/find/fastops.cmake +++ b/cmake/find/fastops.cmake @@ -1,24 +1,19 @@ if(NOT ARCH_ARM AND NOT OS_FREEBSD AND NOT OS_DARWIN) option(ENABLE_FASTOPS "Enable fast vectorized mathematical functions library by Mikhail Parakhin" ${ENABLE_LIBRARIES}) -elseif(ENABLE_FASTOPS) - message (${RECONFIGURE_MESSAGE_LEVEL} "Fastops library is not supported on ARM, FreeBSD and Darwin") endif() -if(NOT ENABLE_FASTOPS) +if(ENABLE_FASTOPS) + if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/fastops/fastops/fastops.h") + message(WARNING "submodule contrib/fastops is missing. to fix try run: \n git submodule update --init --recursive") + set(MISSING_INTERNAL_FASTOPS_LIBRARY 1) + endif() + if(NOT MISSING_INTERNAL_FASTOPS_LIBRARY) + set(USE_FASTOPS 1) + set(FASTOPS_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/fastops/) + set(FASTOPS_LIBRARY fastops) + endif() +else() set(USE_FASTOPS 0) - return() -endif() - -if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/fastops/fastops/fastops.h") - message(WARNING "submodule contrib/fastops is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal fastops library") - set(MISSING_INTERNAL_FASTOPS_LIBRARY 1) -endif() - -if(NOT MISSING_INTERNAL_FASTOPS_LIBRARY) - set(USE_FASTOPS 1) - set(FASTOPS_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/fastops/) - set(FASTOPS_LIBRARY fastops) endif() message(STATUS "Using fastops=${USE_FASTOPS}: ${FASTOPS_INCLUDE_DIR} : ${FASTOPS_LIBRARY}") diff --git a/cmake/find/gperf.cmake b/cmake/find/gperf.cmake index 9b806598c57..451f5c21571 100644 --- a/cmake/find/gperf.cmake +++ b/cmake/find/gperf.cmake @@ -1,11 +1,8 @@ -if(NOT DEFINED ENABLE_GPERF OR ENABLE_GPERF) - # Check if gperf was installed - find_program(GPERF gperf) - if(GPERF) - option(ENABLE_GPERF "Use gperf function hash generator tool" ${ENABLE_LIBRARIES}) - endif() +# Check if gperf was installed +find_program(GPERF gperf) +if(GPERF) + option(ENABLE_GPERF "Use gperf function hash generator tool" ${ENABLE_LIBRARIES}) endif() - if (ENABLE_GPERF) if(NOT GPERF) message(FATAL_ERROR "Could not find the program gperf") diff --git a/cmake/find/grpc.cmake b/cmake/find/grpc.cmake index 216f8ae1878..0019dbd5eed 100644 --- a/cmake/find/grpc.cmake +++ b/cmake/find/grpc.cmake @@ -1,38 +1,26 @@ option (ENABLE_GRPC "Use gRPC" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_GRPC) +if (ENABLE_GRPC) + option (USE_INTERNAL_GRPC_LIBRARY "Set to FALSE to use system gRPC library instead of bundled" ${NOT_UNBUNDLED}) + if (USE_INTERNAL_GRPC_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal gRPC library with ENABLE_GRPC=OFF") - endif() - return() -endif() - -option (USE_INTERNAL_GRPC_LIBRARY "Set to FALSE to use system gRPC library instead of bundled" ${NOT_UNBUNDLED}) - -if (NOT USE_INTERNAL_GRPC_LIBRARY) - find_package(grpc) - if (GRPC_INCLUDE_DIR AND GRPC_LIBRARY) - set (USE_GRPC ON) + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/grpc/include/grpc++/grpc++.h") + message(WARNING "submodule contrib/grpc is missing. To fix try run: \n git submodule update --init --recursive") + set (USE_INTERNAL_GRPC_LIBRARY OFF) + elif (NOT USE_PROTOBUF) + message(WARNING "gRPC requires protobuf which is disabled") + set (USE_INTERNAL_GRPC_LIBRARY OFF) + else() + set (GRPC_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/grpc/include") + set (GRPC_LIBRARY "libgrpc++") + set (USE_GRPC ON) + endif() else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system gRPC") + find_package(grpc) + if (GRPC_INCLUDE_DIR AND GRPC_LIBRARY) + set (USE_GRPC ON) + endif() endif() endif() -if (NOT USE_GRPC) - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/grpc/include/grpc++/grpc++.h") - message (WARNING "submodule contrib/grpc is missing. To fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal gRPC") - set (USE_INTERNAL_GRPC_LIBRARY OFF) - elseif (NOT USE_PROTOBUF) - message (WARNING "gRPC requires protobuf which is disabled") - message (${RECONFIGURE_MESSAGE_LEVEL} "Will not use internal gRPC without protobuf") - set (USE_INTERNAL_GRPC_LIBRARY OFF) - else() - set (GRPC_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/grpc/include") - set (GRPC_LIBRARY "libgrpc++") - set (USE_GRPC ON) - set (USE_INTERNAL_GRPC_LIBRARY ON) - endif() -endif() - -message (STATUS "Using gRPC=${USE_GRPC}: ${GRPC_INCLUDE_DIR} : ${GRPC_LIBRARY}") +message(STATUS "Using gRPC=${USE_GRPC}: ${GRPC_INCLUDE_DIR} : ${GRPC_LIBRARY}") diff --git a/cmake/find/gtest.cmake b/cmake/find/gtest.cmake index 36e45a1381e..b41c4cc0af8 100644 --- a/cmake/find/gtest.cmake +++ b/cmake/find/gtest.cmake @@ -1,30 +1,22 @@ option (ENABLE_GTEST_LIBRARY "Enable gtest library" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_GTEST_LIBRARY) - if(USE_INTERNAL_GTEST_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal Google Test when ENABLE_GTEST_LIBRARY=OFF") - endif() - return() -endif() +if (ENABLE_GTEST_LIBRARY) option (USE_INTERNAL_GTEST_LIBRARY "Set to FALSE to use system Google Test instead of bundled" ${NOT_UNBUNDLED}) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest/CMakeLists.txt") if (USE_INTERNAL_GTEST_LIBRARY) message (WARNING "submodule contrib/googletest is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal gtest") set (USE_INTERNAL_GTEST_LIBRARY 0) endif () set (MISSING_INTERNAL_GTEST_LIBRARY 1) endif () + if(NOT USE_INTERNAL_GTEST_LIBRARY) # TODO: autodetect of GTEST_SRC_DIR by EXISTS /usr/src/googletest/CMakeLists.txt if(NOT GTEST_SRC_DIR) find_package(GTest) - if (NOT GTEST_INCLUDE_DIRS) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system Google Test") - endif() endif() endif() @@ -34,13 +26,12 @@ if (NOT GTEST_SRC_DIR AND NOT GTEST_INCLUDE_DIRS AND NOT MISSING_INTERNAL_GTEST_ set (GTEST_LIBRARIES gtest) set (GTEST_BOTH_LIBRARIES ${GTEST_MAIN_LIBRARIES} ${GTEST_LIBRARIES}) set (GTEST_INCLUDE_DIRS ${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest) -elseif(USE_INTERNAL_GTEST_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Wouldn't use internal Google Test library") - set (USE_INTERNAL_GTEST_LIBRARY 0) endif () if((GTEST_INCLUDE_DIRS AND GTEST_BOTH_LIBRARIES) OR GTEST_SRC_DIR) set(USE_GTEST 1) endif() +endif() + message (STATUS "Using gtest=${USE_GTEST}: ${GTEST_INCLUDE_DIRS} : ${GTEST_BOTH_LIBRARIES} : ${GTEST_SRC_DIR}") diff --git a/cmake/find/h3.cmake b/cmake/find/h3.cmake index 7ca070b45fe..e01f0269507 100644 --- a/cmake/find/h3.cmake +++ b/cmake/find/h3.cmake @@ -1,38 +1,28 @@ option (ENABLE_H3 "Enable H3" ${ENABLE_LIBRARIES}) -if(NOT ENABLE_H3) - if(USE_INTERNAL_H3_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal h3 library with ENABLE_H3=OFF") - endif () - return() -endif() +if (ENABLE_H3) -option(USE_INTERNAL_H3_LIBRARY "Set to FALSE to use system h3 library instead of bundled" ${NOT_UNBUNDLED}) +option (USE_INTERNAL_H3_LIBRARY "Set to FALSE to use system h3 library instead of bundled" ${NOT_UNBUNDLED}) if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/h3/src/h3lib/include/h3Index.h") if(USE_INTERNAL_H3_LIBRARY) - message(WARNING "submodule contrib/h3 is missing. to fix try run: \n git submodule update --init --recursive") - message(${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal h3 library") - set(USE_INTERNAL_H3_LIBRARY 0) + message(WARNING "submodule contrib/h3 is missing. to fix try run: \n git submodule update --init --recursive") endif() set(MISSING_INTERNAL_H3_LIBRARY 1) + set(USE_INTERNAL_H3_LIBRARY 0) endif() -if(NOT USE_INTERNAL_H3_LIBRARY) - find_library(H3_LIBRARY h3) - find_path(H3_INCLUDE_DIR NAMES h3/h3api.h PATHS ${H3_INCLUDE_PATHS}) - - if(NOT H3_LIBRARY OR NOT H3_INCLUDE_DIR) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system h3 library") - endif() -endif() +if (USE_INTERNAL_H3_LIBRARY) + set (H3_LIBRARY h3) + set (H3_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/h3/src/h3lib/include) +elseif (NOT MISSING_INTERNAL_H3_LIBRARY) + find_library (H3_LIBRARY h3) + find_path (H3_INCLUDE_DIR NAMES h3/h3api.h PATHS ${H3_INCLUDE_PATHS}) +endif () if (H3_LIBRARY AND H3_INCLUDE_DIR) set (USE_H3 1) -elseif(NOT MISSING_INTERNAL_H3_LIBRARY) - set (H3_LIBRARY h3) - set (H3_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/h3/src/h3lib/include") - set (USE_H3 1) - set (USE_INTERNAL_H3_LIBRARY 1) -endif() +endif () + +endif () message (STATUS "Using h3=${USE_H3}: ${H3_INCLUDE_DIR} : ${H3_LIBRARY}") diff --git a/cmake/find/hdfs3.cmake b/cmake/find/hdfs3.cmake index e8e9fb359c9..6a0d01d2245 100644 --- a/cmake/find/hdfs3.cmake +++ b/cmake/find/hdfs3.cmake @@ -1,33 +1,21 @@ if(NOT ARCH_ARM AND NOT OS_FREEBSD AND NOT APPLE AND USE_PROTOBUF) option(ENABLE_HDFS "Enable HDFS" ${ENABLE_LIBRARIES}) -elseif(ENABLE_HDFS OR USE_INTERNAL_HDFS3_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use HDFS3 with current configuration") -endif() - -if(NOT ENABLE_HDFS) - if(USE_INTERNAL_HDFS3_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal HDFS3 library with ENABLE_HDFS3=OFF") - endif() - return() endif() +if(ENABLE_HDFS) option(USE_INTERNAL_HDFS3_LIBRARY "Set to FALSE to use system HDFS3 instead of bundled" ${NOT_UNBUNDLED}) if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libhdfs3/include/hdfs/hdfs.h") if(USE_INTERNAL_HDFS3_LIBRARY) - message(WARNING "submodule contrib/libhdfs3 is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal HDFS3 library") - set(USE_INTERNAL_HDFS3_LIBRARY 0) + message(WARNING "submodule contrib/libhdfs3 is missing. to fix try run: \n git submodule update --init --recursive") endif() set(MISSING_INTERNAL_HDFS3_LIBRARY 1) + set(USE_INTERNAL_HDFS3_LIBRARY 0) endif() if(NOT USE_INTERNAL_HDFS3_LIBRARY) find_library(HDFS3_LIBRARY hdfs3) find_path(HDFS3_INCLUDE_DIR NAMES hdfs/hdfs.h PATHS ${HDFS3_INCLUDE_PATHS}) - if(NOT HDFS3_LIBRARY OR NOT HDFS3_INCLUDE_DIR) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot find system HDFS3 library") - endif() endif() if(HDFS3_LIBRARY AND HDFS3_INCLUDE_DIR) @@ -38,7 +26,9 @@ elseif(NOT MISSING_INTERNAL_HDFS3_LIBRARY AND LIBGSASL_LIBRARY AND LIBXML2_LIBRA set(USE_INTERNAL_HDFS3_LIBRARY 1) set(USE_HDFS 1) else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannout enable HDFS3") + set(USE_INTERNAL_HDFS3_LIBRARY 0) +endif() + endif() message(STATUS "Using hdfs3=${USE_HDFS}: ${HDFS3_INCLUDE_DIR} : ${HDFS3_LIBRARY}") diff --git a/cmake/find/icu.cmake b/cmake/find/icu.cmake index 40fb391656d..7beb25626b9 100644 --- a/cmake/find/icu.cmake +++ b/cmake/find/icu.cmake @@ -4,20 +4,13 @@ else () option(ENABLE_ICU "Enable ICU" 0) endif () -if (NOT ENABLE_ICU) - if(USE_INTERNAL_ICU_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal icu library with ENABLE_ICU=OFF") - endif() - message(STATUS "Build without ICU (support for collations and charset conversion functions will be disabled)") - return() -endif() +if (ENABLE_ICU) option (USE_INTERNAL_ICU_LIBRARY "Set to FALSE to use system ICU library instead of bundled" ${NOT_UNBUNDLED}) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/icu/icu4c/LICENSE") if (USE_INTERNAL_ICU_LIBRARY) message (WARNING "submodule contrib/icu is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal ICU") set (USE_INTERNAL_ICU_LIBRARY 0) endif () set (MISSING_INTERNAL_ICU_LIBRARY 1) @@ -31,8 +24,6 @@ if(NOT USE_INTERNAL_ICU_LIBRARY) #set (ICU_LIBRARIES ${ICU_I18N_LIBRARY} ${ICU_UC_LIBRARY} ${ICU_DATA_LIBRARY} CACHE STRING "") if(ICU_FOUND) set(USE_ICU 1) - else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system ICU") endif() endif() @@ -44,6 +35,8 @@ elseif (NOT MISSING_INTERNAL_ICU_LIBRARY) set (USE_ICU 1) endif () +endif() + if(USE_ICU) message(STATUS "Using icu=${USE_ICU}: ${ICU_INCLUDE_DIR} : ${ICU_LIBRARIES}") else() diff --git a/cmake/find/ldap.cmake b/cmake/find/ldap.cmake index 3f7871617d4..99c9007d6b5 100644 --- a/cmake/find/ldap.cmake +++ b/cmake/find/ldap.cmake @@ -1,97 +1,84 @@ option (ENABLE_LDAP "Enable LDAP" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_LDAP) - if(USE_INTERNAL_LDAP_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal LDAP library with ENABLE_LDAP=OFF") - endif () - return() -endif() +if (ENABLE_LDAP) + option (USE_INTERNAL_LDAP_LIBRARY "Set to FALSE to use system *LDAP library instead of bundled" ${NOT_UNBUNDLED}) -option (USE_INTERNAL_LDAP_LIBRARY "Set to FALSE to use system *LDAP library instead of bundled" ${NOT_UNBUNDLED}) - -if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/openldap/README") - if (USE_INTERNAL_LDAP_LIBRARY) - message (WARNING "Submodule contrib/openldap is missing. To fix try running:\n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal LDAP library") - endif () - - set (USE_INTERNAL_LDAP_LIBRARY 0) - set (MISSING_INTERNAL_LDAP_LIBRARY 1) -endif () - -set (OPENLDAP_USE_STATIC_LIBS ${USE_STATIC_LIBRARIES}) -set (OPENLDAP_USE_REENTRANT_LIBS 1) - -if (NOT USE_INTERNAL_LDAP_LIBRARY) - if (OPENLDAP_USE_STATIC_LIBS) - message (WARNING "Unable to use external static OpenLDAP libraries, falling back to the bundled version.") - message (${RECONFIGURE_MESSAGE_LEVEL} "Unable to use external OpenLDAP") - set (USE_INTERNAL_LDAP_LIBRARY 1) - else () - if (APPLE AND NOT OPENLDAP_ROOT_DIR) - set (OPENLDAP_ROOT_DIR "/usr/local/opt/openldap") + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/openldap/README") + if (USE_INTERNAL_LDAP_LIBRARY) + message (WARNING "Submodule contrib/openldap is missing. To fix try running:\n git submodule update --init --recursive") endif () - find_package (OpenLDAP) - - if (NOT OPENLDAP_FOUND) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system OpenLDAP") - endif() - endif () -endif () - -if (NOT OPENLDAP_FOUND AND NOT MISSING_INTERNAL_LDAP_LIBRARY) - string (TOLOWER "${CMAKE_SYSTEM_NAME}" _system_name) - string (TOLOWER "${CMAKE_SYSTEM_PROCESSOR}" _system_processor) - - if ( - "${_system_processor}" STREQUAL "amd64" OR - "${_system_processor}" STREQUAL "x64" - ) - set (_system_processor "x86_64") - elseif ( - "${_system_processor}" STREQUAL "arm64" - ) - set (_system_processor "aarch64") + set (USE_INTERNAL_LDAP_LIBRARY 0) + set (MISSING_INTERNAL_LDAP_LIBRARY 1) endif () - if ( - ( "${_system_name}" STREQUAL "linux" AND "${_system_processor}" STREQUAL "x86_64" ) OR - ( "${_system_name}" STREQUAL "linux" AND "${_system_processor}" STREQUAL "aarch64" ) OR - ( "${_system_name}" STREQUAL "freebsd" AND "${_system_processor}" STREQUAL "x86_64" ) OR - ( "${_system_name}" STREQUAL "darwin" AND "${_system_processor}" STREQUAL "x86_64" ) - ) - set (_ldap_supported_platform TRUE) - endif () + set (OPENLDAP_USE_STATIC_LIBS ${USE_STATIC_LIBRARIES}) + set (OPENLDAP_USE_REENTRANT_LIBS 1) - if (NOT _ldap_supported_platform) - message (WARNING "LDAP support using the bundled library is not implemented for ${CMAKE_SYSTEM_NAME} ${CMAKE_SYSTEM_PROCESSOR} platform.") - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot enable LDAP support") - elseif (NOT USE_SSL) - message (WARNING "LDAP support using the bundled library is not possible if SSL is not used.") - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot enable LDAP support") - else () - set (USE_INTERNAL_LDAP_LIBRARY 1) - set (OPENLDAP_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/openldap") - set (OPENLDAP_INCLUDE_DIRS - "${ClickHouse_SOURCE_DIR}/contrib/openldap-cmake/${_system_name}_${_system_processor}/include" - "${ClickHouse_SOURCE_DIR}/contrib/openldap/include" - ) - # Below, 'ldap'/'ldap_r' and 'lber' will be resolved to - # the targets defined in contrib/openldap-cmake/CMakeLists.txt - if (OPENLDAP_USE_REENTRANT_LIBS) - set (OPENLDAP_LDAP_LIBRARY "ldap_r") + if (NOT USE_INTERNAL_LDAP_LIBRARY) + if (OPENLDAP_USE_STATIC_LIBS) + message (WARNING "Unable to use external static OpenLDAP libraries, falling back to the bundled version.") + set (USE_INTERNAL_LDAP_LIBRARY 1) else () - set (OPENLDAP_LDAP_LIBRARY "ldap") - endif() - set (OPENLDAP_LBER_LIBRARY "lber") - set (OPENLDAP_LIBRARIES ${OPENLDAP_LDAP_LIBRARY} ${OPENLDAP_LBER_LIBRARY}) - set (OPENLDAP_FOUND 1) - endif () -endif () + if (APPLE AND NOT OPENLDAP_ROOT_DIR) + set (OPENLDAP_ROOT_DIR "/usr/local/opt/openldap") + endif () -if (OPENLDAP_FOUND) - set (USE_LDAP 1) + find_package (OpenLDAP) + endif () + endif () + + if (NOT OPENLDAP_FOUND AND NOT MISSING_INTERNAL_LDAP_LIBRARY) + string (TOLOWER "${CMAKE_SYSTEM_NAME}" _system_name) + string (TOLOWER "${CMAKE_SYSTEM_PROCESSOR}" _system_processor) + + if ( + "${_system_processor}" STREQUAL "amd64" OR + "${_system_processor}" STREQUAL "x64" + ) + set (_system_processor "x86_64") + elseif ( + "${_system_processor}" STREQUAL "arm64" + ) + set (_system_processor "aarch64") + endif () + + if ( + ( "${_system_name}" STREQUAL "linux" AND "${_system_processor}" STREQUAL "x86_64" ) OR + ( "${_system_name}" STREQUAL "linux" AND "${_system_processor}" STREQUAL "aarch64" ) OR + ( "${_system_name}" STREQUAL "freebsd" AND "${_system_processor}" STREQUAL "x86_64" ) OR + ( "${_system_name}" STREQUAL "darwin" AND "${_system_processor}" STREQUAL "x86_64" ) + ) + set (_ldap_supported_platform TRUE) + endif () + + if (NOT _ldap_supported_platform) + message (WARNING "LDAP support using the bundled library is not implemented for ${CMAKE_SYSTEM_NAME} ${CMAKE_SYSTEM_PROCESSOR} platform.") + elseif (NOT USE_SSL) + message (WARNING "LDAP support using the bundled library is not possible if SSL is not used.") + else () + set (USE_INTERNAL_LDAP_LIBRARY 1) + set (OPENLDAP_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/openldap") + set (OPENLDAP_INCLUDE_DIRS + "${ClickHouse_SOURCE_DIR}/contrib/openldap-cmake/${_system_name}_${_system_processor}/include" + "${ClickHouse_SOURCE_DIR}/contrib/openldap/include" + ) + # Below, 'ldap'/'ldap_r' and 'lber' will be resolved to + # the targets defined in contrib/openldap-cmake/CMakeLists.txt + if (OPENLDAP_USE_REENTRANT_LIBS) + set (OPENLDAP_LDAP_LIBRARY "ldap_r") + else () + set (OPENLDAP_LDAP_LIBRARY "ldap") + endif() + set (OPENLDAP_LBER_LIBRARY "lber") + set (OPENLDAP_LIBRARIES ${OPENLDAP_LDAP_LIBRARY} ${OPENLDAP_LBER_LIBRARY}) + set (OPENLDAP_FOUND 1) + endif () + endif () + + if (OPENLDAP_FOUND) + set (USE_LDAP 1) + endif () endif () message (STATUS "Using ldap=${USE_LDAP}: ${OPENLDAP_INCLUDE_DIRS} : ${OPENLDAP_LIBRARIES}") diff --git a/cmake/find/libgsasl.cmake b/cmake/find/libgsasl.cmake index 3b580a4ddb2..e9c45a09010 100644 --- a/cmake/find/libgsasl.cmake +++ b/cmake/find/libgsasl.cmake @@ -1,18 +1,12 @@ option(ENABLE_GSASL_LIBRARY "Enable gsasl library" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_GSASL_LIBRARY) - if(USE_INTERNAL_LIBGSASL_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal libgsasl library with ENABLE_GSASL_LIBRARY=OFF") - endif() - return() -endif() +if (ENABLE_GSASL_LIBRARY) option (USE_INTERNAL_LIBGSASL_LIBRARY "Set to FALSE to use system libgsasl library instead of bundled" ${NOT_UNBUNDLED}) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src/gsasl.h") if (USE_INTERNAL_LIBGSASL_LIBRARY) message (WARNING "submodule contrib/libgsasl is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libgsasl") set (USE_INTERNAL_LIBGSASL_LIBRARY 0) endif () set (MISSING_INTERNAL_LIBGSASL_LIBRARY 1) @@ -20,15 +14,12 @@ endif () if (NOT USE_INTERNAL_LIBGSASL_LIBRARY) find_library (LIBGSASL_LIBRARY gsasl) - find_path (LIBGSASL_INCLUDE_DIR NAMES gsasl.h PATHS ${LIBGSASL_INCLUDE_PATHS}) - if (NOT LIBGSASL_LIBRARY OR NOT LIBGSASL_INCLUDE_DIR) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system libgsasl") - endif () + find_path (LIBGSASL_INCLUDE_DIR NAMES gsasl.h PATHS ${LIBGSASL_INCLUDE_PATHS}) endif () if (LIBGSASL_LIBRARY AND LIBGSASL_INCLUDE_DIR) elseif (NOT MISSING_INTERNAL_LIBGSASL_LIBRARY) - set (LIBGSASL_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src" "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/linux_x86_64/include") + set (LIBGSASL_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src ${ClickHouse_SOURCE_DIR}/contrib/libgsasl/linux_x86_64/include) set (USE_INTERNAL_LIBGSASL_LIBRARY 1) set (LIBGSASL_LIBRARY libgsasl) endif () @@ -37,4 +28,6 @@ if(LIBGSASL_LIBRARY AND LIBGSASL_INCLUDE_DIR) set (USE_LIBGSASL 1) endif() +endif() + message (STATUS "Using libgsasl=${USE_LIBGSASL}: ${LIBGSASL_INCLUDE_DIR} : ${LIBGSASL_LIBRARY}") diff --git a/cmake/find/libxml2.cmake b/cmake/find/libxml2.cmake index 04609e497a8..73732e139c5 100644 --- a/cmake/find/libxml2.cmake +++ b/cmake/find/libxml2.cmake @@ -3,7 +3,6 @@ option (USE_INTERNAL_LIBXML2_LIBRARY "Set to FALSE to use system libxml2 library if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libxml2/libxml.h") if (USE_INTERNAL_LIBXML2_LIBRARY) message (WARNING "submodule contrib/libxml2 is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libxml") set (USE_INTERNAL_LIBXML2_LIBRARY 0) endif () set (MISSING_INTERNAL_LIBXML2_LIBRARY 1) @@ -13,10 +12,6 @@ if (NOT USE_INTERNAL_LIBXML2_LIBRARY) find_package (LibXml2) #find_library (LIBXML2_LIBRARY libxml2) #find_path (LIBXML2_INCLUDE_DIR NAMES libxml.h PATHS ${LIBXML2_INCLUDE_PATHS}) - - if (NOT LIBXML2_LIBRARY OR NOT LIBXML2_INCLUDE_DIR) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system libxml2") - endif () endif () if (LIBXML2_LIBRARY AND LIBXML2_INCLUDE_DIR) diff --git a/cmake/find/llvm.cmake b/cmake/find/llvm.cmake index e46b8c9335e..7cb67d1a990 100644 --- a/cmake/find/llvm.cmake +++ b/cmake/find/llvm.cmake @@ -2,89 +2,78 @@ if (NOT APPLE) option (ENABLE_EMBEDDED_COMPILER "Set to TRUE to enable support for 'compile_expressions' option for query execution" ${ENABLE_LIBRARIES}) option (USE_INTERNAL_LLVM_LIBRARY "Use bundled or system LLVM library." ${NOT_UNBUNDLED}) -elseif(ENABLE_EMBEDDED_COMPILER OR USE_INTERNAL_LLVM_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Embedded compiler is not currently supported on Apple") endif () -if (NOT ENABLE_EMBEDDED_COMPILER) - if(USE_INTERNAL_LLVM_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal LLVM library with ENABLE_EMBEDDED_COMPILER=OFF") - endif() - return() -endif() +if (ENABLE_EMBEDDED_COMPILER) + if (USE_INTERNAL_LLVM_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/llvm/llvm/CMakeLists.txt") + message (WARNING "submodule contrib/llvm is missing. to fix try run: \n git submodule update --init --recursive") + set (USE_INTERNAL_LLVM_LIBRARY 0) + endif () -if (USE_INTERNAL_LLVM_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/llvm/llvm/CMakeLists.txt") - message (WARNING "submodule contrib/llvm is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't fidd internal LLVM library") - set (USE_INTERNAL_LLVM_LIBRARY 0) -endif () + if (NOT USE_INTERNAL_LLVM_LIBRARY) + set (LLVM_PATHS "/usr/local/lib/llvm") -if (NOT USE_INTERNAL_LLVM_LIBRARY) - set (LLVM_PATHS "/usr/local/lib/llvm") + foreach(llvm_v 9 8) + if (NOT LLVM_FOUND) + find_package (LLVM ${llvm_v} CONFIG PATHS ${LLVM_PATHS}) + endif () + endforeach () - foreach(llvm_v 9 8) - if (NOT LLVM_FOUND) - find_package (LLVM ${llvm_v} CONFIG PATHS ${LLVM_PATHS}) + if (LLVM_FOUND) + # Remove dynamically-linked zlib and libedit from LLVM's dependencies: + set_target_properties(LLVMSupport PROPERTIES INTERFACE_LINK_LIBRARIES "-lpthread;LLVMDemangle;${ZLIB_LIBRARIES}") + set_target_properties(LLVMLineEditor PROPERTIES INTERFACE_LINK_LIBRARIES "LLVMSupport") + + option(LLVM_HAS_RTTI "Enable if LLVM was build with RTTI enabled" ON) + set (USE_EMBEDDED_COMPILER 1) + else() + set (USE_EMBEDDED_COMPILER 0) + endif() + + if (LLVM_FOUND AND OS_LINUX AND USE_LIBCXX) + message(WARNING "Option USE_INTERNAL_LLVM_LIBRARY is not set but the LLVM library from OS packages in Linux is incompatible with libc++ ABI. LLVM Will be disabled.") + set (LLVM_FOUND 0) + set (USE_EMBEDDED_COMPILER 0) endif () - endforeach () + else() + if (CMAKE_CURRENT_SOURCE_DIR STREQUAL CMAKE_CURRENT_BINARY_DIR) + message(WARNING "Option ENABLE_EMBEDDED_COMPILER is set but LLVM library cannot build if build directory is the same as source directory.") + set (LLVM_FOUND 0) + set (USE_EMBEDDED_COMPILER 0) + elseif (SPLIT_SHARED_LIBRARIES) + # llvm-tablegen cannot find shared libraries that we build. Probably can be easily fixed. + message(WARNING "Option ENABLE_EMBEDDED_COMPILER is not compatible with SPLIT_SHARED_LIBRARIES. Build of LLVM will be disabled.") + set (LLVM_FOUND 0) + set (USE_EMBEDDED_COMPILER 0) + elseif (NOT ARCH_AMD64) + # It's not supported yet, but you can help. + message(WARNING "Option ENABLE_EMBEDDED_COMPILER is only available for x86_64. Build of LLVM will be disabled.") + set (LLVM_FOUND 0) + set (USE_EMBEDDED_COMPILER 0) + elseif (SANITIZE STREQUAL "undefined") + # llvm-tblgen, that is used during LLVM build, doesn't work with UBSan. + message(WARNING "Option ENABLE_EMBEDDED_COMPILER does not work with UBSan, because 'llvm-tblgen' tool from LLVM has undefined behaviour. Build of LLVM will be disabled.") + set (LLVM_FOUND 0) + set (USE_EMBEDDED_COMPILER 0) + else () + set (LLVM_FOUND 1) + set (USE_EMBEDDED_COMPILER 1) + set (LLVM_VERSION "9.0.0bundled") + set (LLVM_INCLUDE_DIRS + ${ClickHouse_SOURCE_DIR}/contrib/llvm/llvm/include + ${ClickHouse_BINARY_DIR}/contrib/llvm/llvm/include + ) + set (LLVM_LIBRARY_DIRS ${ClickHouse_BINARY_DIR}/contrib/llvm/llvm) + endif() + endif() if (LLVM_FOUND) - # Remove dynamically-linked zlib and libedit from LLVM's dependencies: - set_target_properties(LLVMSupport PROPERTIES INTERFACE_LINK_LIBRARIES "-lpthread;LLVMDemangle;${ZLIB_LIBRARIES}") - set_target_properties(LLVMLineEditor PROPERTIES INTERFACE_LINK_LIBRARIES "LLVMSupport") - - option(LLVM_HAS_RTTI "Enable if LLVM was build with RTTI enabled" ON) - set (USE_EMBEDDED_COMPILER 1) - else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system LLVM") - set (USE_EMBEDDED_COMPILER 0) - endif() - - if (LLVM_FOUND AND OS_LINUX AND USE_LIBCXX) - message(WARNING "Option USE_INTERNAL_LLVM_LIBRARY is not set but the LLVM library from OS packages in Linux is incompatible with libc++ ABI. LLVM Will be disabled.") - message (${RECONFIGURE_MESSAGE_LEVEL} "Unsupported LLVM configuration, cannot enable LLVM") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - endif () -else() - if (CMAKE_CURRENT_SOURCE_DIR STREQUAL CMAKE_CURRENT_BINARY_DIR) - message(WARNING "Option ENABLE_EMBEDDED_COMPILER is set but LLVM library cannot build if build directory is the same as source directory.") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - elseif (SPLIT_SHARED_LIBRARIES) - # llvm-tablegen cannot find shared libraries that we build. Probably can be easily fixed. - message(WARNING "Option ENABLE_EMBEDDED_COMPILER is not compatible with SPLIT_SHARED_LIBRARIES. Build of LLVM will be disabled.") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - elseif (NOT ARCH_AMD64) - # It's not supported yet, but you can help. - message(WARNING "Option ENABLE_EMBEDDED_COMPILER is only available for x86_64. Build of LLVM will be disabled.") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - elseif (SANITIZE STREQUAL "undefined") - # llvm-tblgen, that is used during LLVM build, doesn't work with UBSan. - message(WARNING "Option ENABLE_EMBEDDED_COMPILER does not work with UBSan, because 'llvm-tblgen' tool from LLVM has undefined behaviour. Build of LLVM will be disabled.") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - else () - set (LLVM_FOUND 1) - set (USE_EMBEDDED_COMPILER 1) - set (LLVM_VERSION "9.0.0bundled") - set (LLVM_INCLUDE_DIRS - ${ClickHouse_SOURCE_DIR}/contrib/llvm/llvm/include - ${ClickHouse_BINARY_DIR}/contrib/llvm/llvm/include - ) - set (LLVM_LIBRARY_DIRS ${ClickHouse_BINARY_DIR}/contrib/llvm/llvm) + message(STATUS "LLVM include Directory: ${LLVM_INCLUDE_DIRS}") + message(STATUS "LLVM library Directory: ${LLVM_LIBRARY_DIRS}") + message(STATUS "LLVM C++ compiler flags: ${LLVM_CXXFLAGS}") endif() endif() -if (LLVM_FOUND) - message(STATUS "LLVM include Directory: ${LLVM_INCLUDE_DIRS}") - message(STATUS "LLVM library Directory: ${LLVM_LIBRARY_DIRS}") - message(STATUS "LLVM C++ compiler flags: ${LLVM_CXXFLAGS}") -else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't enable LLVM") -endif() # This list was generated by listing all LLVM libraries, compiling the binary and removing all libraries while it still compiles. set (REQUIRED_LLVM_LIBRARIES diff --git a/cmake/find/ltdl.cmake b/cmake/find/ltdl.cmake index b48a3630222..18003618dbd 100644 --- a/cmake/find/ltdl.cmake +++ b/cmake/find/ltdl.cmake @@ -1,5 +1,5 @@ if (ENABLE_ODBC AND NOT USE_INTERNAL_ODBC_LIBRARY) set (LTDL_PATHS "/usr/local/opt/libtool/lib") - find_library (LTDL_LIBRARY ltdl PATHS ${LTDL_PATHS} REQUIRED) + find_library (LTDL_LIBRARY ltdl PATHS ${LTDL_PATHS}) message (STATUS "Using ltdl: ${LTDL_LIBRARY}") endif () diff --git a/cmake/find/msgpack.cmake b/cmake/find/msgpack.cmake index 130aa007ad5..102ea619f6a 100644 --- a/cmake/find/msgpack.cmake +++ b/cmake/find/msgpack.cmake @@ -1,37 +1,27 @@ option (ENABLE_MSGPACK "Enable msgpack library" ${ENABLE_LIBRARIES}) -if(NOT ENABLE_MSGPACK) - if(USE_INTERNAL_MSGPACK_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal msgpack with ENABLE_MSGPACK=OFF") - endif() - return() -endif() +if (ENABLE_MSGPACK) option (USE_INTERNAL_MSGPACK_LIBRARY "Set to FALSE to use system msgpack library instead of bundled" ${NOT_UNBUNDLED}) -if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/msgpack-c/include/msgpack.hpp") - if(USE_INTERNAL_MSGPACK_LIBRARY) +if (USE_INTERNAL_MSGPACK_LIBRARY) + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/msgpack-c/include/msgpack.hpp") message(WARNING "Submodule contrib/msgpack-c is missing. To fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal msgpack") - set(USE_INTERNAL_MSGPACK_LIBRARY 0) + set(USE_INTERNAL_MSGPACK_LIBRARY 0) + set(MISSING_INTERNAL_MSGPACK_LIBRARY 1) endif() - set(MISSING_INTERNAL_MSGPACK_LIBRARY 1) endif() -if(NOT USE_INTERNAL_MSGPACK_LIBRARY) +if (USE_INTERNAL_MSGPACK_LIBRARY) + set(MSGPACK_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/msgpack-c/include) +else() find_path(MSGPACK_INCLUDE_DIR NAMES msgpack.hpp PATHS ${MSGPACK_INCLUDE_PATHS}) - if(NOT MSGPACK_INCLUDE_DIR) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system msgpack") - endif() -endif() - -if(NOT MSGPACK_INCLUDE_DIR AND NOT MISSING_INTERNAL_MSGPACK_LIBRARY) - set(MSGPACK_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/msgpack-c/include") - set(USE_INTERNAL_MSGPACK_LIBRARY 1) endif() if (MSGPACK_INCLUDE_DIR) set(USE_MSGPACK 1) endif() +endif() + message(STATUS "Using msgpack=${USE_MSGPACK}: ${MSGPACK_INCLUDE_DIR}") diff --git a/cmake/find/mysqlclient.cmake b/cmake/find/mysqlclient.cmake index b0bfd02fabc..886c5b9bb59 100644 --- a/cmake/find/mysqlclient.cmake +++ b/cmake/find/mysqlclient.cmake @@ -4,71 +4,56 @@ else () option(ENABLE_MYSQL "Enable MySQL" FALSE) endif () -if(NOT ENABLE_MYSQL) - if (USE_INTERNAL_MYSQL_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal mysql library with ENABLE_MYSQL=OFF") - endif () - message (STATUS "Build without mysqlclient (support for MYSQL dictionary source will be disabled)") - return() -endif() +if(ENABLE_MYSQL) + option(USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" ${NOT_UNBUNDLED}) -option(USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" ${NOT_UNBUNDLED}) - -if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README") - if(USE_INTERNAL_MYSQL_LIBRARY) + if(USE_INTERNAL_MYSQL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README") message(WARNING "submodule contrib/mariadb-connector-c is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal mysql library") set(USE_INTERNAL_MYSQL_LIBRARY 0) endif() - set(MISSING_INTERNAL_MYSQL_LIBRARY 1) -endif() -if (NOT USE_INTERNAL_MYSQL_LIBRARY) - set (MYSQL_LIB_PATHS - "/usr/local/opt/mysql/lib" - "/usr/local/lib" - "/usr/local/lib64" - "/usr/local/lib/mariadb" # macos brew mariadb-connector-c - "/usr/mysql/lib" - "/usr/mysql/lib64" - "/usr/lib" - "/usr/lib64" - "/lib" - "/lib64") - - set (MYSQL_INCLUDE_PATHS - "/usr/local/opt/mysql/include" - "/usr/mysql/include" - "/usr/local/include" - "/usr/include") - - find_path (MYSQL_INCLUDE_DIR NAMES mysql/mysql.h mariadb/mysql.h PATHS ${MYSQL_INCLUDE_PATHS} PATH_SUFFIXES mysql) - - if (USE_STATIC_LIBRARIES) - find_library (STATIC_MYSQLCLIENT_LIB NAMES mariadbclient mysqlclient PATHS ${MYSQL_LIB_PATHS} PATH_SUFFIXES mysql) - else () - find_library (MYSQLCLIENT_LIBRARIES NAMES mariadb mariadbclient mysqlclient PATHS ${MYSQL_LIB_PATHS} PATH_SUFFIXES mysql) - endif () - - if (MYSQL_INCLUDE_DIR AND (STATIC_MYSQLCLIENT_LIB OR MYSQLCLIENT_LIBRARIES)) + if (USE_INTERNAL_MYSQL_LIBRARY) + set (MYSQLCLIENT_LIBRARIES mariadbclient) set (USE_MYSQL 1) set (MYSQLXX_LIBRARY mysqlxx) - if (APPLE) - # /usr/local/include/mysql/mysql_com.h:1011:10: fatal error: mysql/udf_registration_types.h: No such file or directory - set(MYSQL_INCLUDE_DIR ${MYSQL_INCLUDE_DIR} ${MYSQL_INCLUDE_DIR}/mysql) - endif () else () - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system mysql library") + set (MYSQL_LIB_PATHS + "/usr/local/opt/mysql/lib" + "/usr/local/lib" + "/usr/local/lib64" + "/usr/local/lib/mariadb" # macos brew mariadb-connector-c + "/usr/mysql/lib" + "/usr/mysql/lib64" + "/usr/lib" + "/usr/lib64" + "/lib" + "/lib64") + + set (MYSQL_INCLUDE_PATHS + "/usr/local/opt/mysql/include" + "/usr/mysql/include" + "/usr/local/include" + "/usr/include") + + find_path (MYSQL_INCLUDE_DIR NAMES mysql/mysql.h mariadb/mysql.h PATHS ${MYSQL_INCLUDE_PATHS} PATH_SUFFIXES mysql) + + if (USE_STATIC_LIBRARIES) + find_library (STATIC_MYSQLCLIENT_LIB NAMES mariadbclient mysqlclient PATHS ${MYSQL_LIB_PATHS} PATH_SUFFIXES mysql) + else () + find_library (MYSQLCLIENT_LIBRARIES NAMES mariadb mariadbclient mysqlclient PATHS ${MYSQL_LIB_PATHS} PATH_SUFFIXES mysql) + endif () + + if (MYSQL_INCLUDE_DIR AND (STATIC_MYSQLCLIENT_LIB OR MYSQLCLIENT_LIBRARIES)) + set (USE_MYSQL 1) + set (MYSQLXX_LIBRARY mysqlxx) + if (APPLE) + # /usr/local/include/mysql/mysql_com.h:1011:10: fatal error: mysql/udf_registration_types.h: No such file or directory + set(MYSQL_INCLUDE_DIR ${MYSQL_INCLUDE_DIR} ${MYSQL_INCLUDE_DIR}/mysql) + endif () + endif () endif () endif () -if (NOT USE_MYSQL AND NOT MISSING_INTERNAL_MYSQL_LIBRARY) - set (MYSQLCLIENT_LIBRARIES mariadbclient) - set (MYSQLXX_LIBRARY mysqlxx) - set (USE_MYSQL 1) - set (USE_INTERNAL_MYSQL_LIBRARY 1) -endif() - if (USE_MYSQL) message (STATUS "Using mysqlclient=${USE_MYSQL}: ${MYSQL_INCLUDE_DIR} : ${MYSQLCLIENT_LIBRARIES}; staticlib=${STATIC_MYSQLCLIENT_LIB}") else () diff --git a/cmake/find/opencl.cmake b/cmake/find/opencl.cmake index 2b0cc7c5dd4..0f307350cb8 100644 --- a/cmake/find/opencl.cmake +++ b/cmake/find/opencl.cmake @@ -3,14 +3,12 @@ if(0) option(ENABLE_OPENCL "Enable OpenCL support" ${ENABLE_LIBRARIES}) endif() -if(NOT ENABLE_OPENCL) - return() -endif() +if(ENABLE_OPENCL) # Intel OpenCl driver: sudo apt install intel-opencl-icd # @sa https://github.com/intel/compute-runtime/releases -# OpenCL applications should link with ICD loader +# OpenCL applications should link wiht ICD loader # sudo apt install opencl-headers ocl-icd-libopencl1 # sudo ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so # TODO: add https://github.com/OCL-dev/ocl-icd as submodule instead @@ -18,8 +16,8 @@ endif() find_package(OpenCL) if(OpenCL_FOUND) set(USE_OPENCL 1) -else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't enable OpenCL support") +endif() + endif() message(STATUS "Using opencl=${USE_OPENCL}: ${OpenCL_INCLUDE_DIRS} : ${OpenCL_LIBRARIES}") diff --git a/cmake/find/orc.cmake b/cmake/find/orc.cmake index c9474a095c4..26253687c80 100644 --- a/cmake/find/orc.cmake +++ b/cmake/find/orc.cmake @@ -1,20 +1,12 @@ option (ENABLE_ORC "Enable ORC" ${ENABLE_LIBRARIES}) -if(NOT ENABLE_ORC) - if(USE_INTERNAL_ORC_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal ORC library with ENABLE_ORD=OFF") - endif() - return() -endif() - -option(USE_INTERNAL_ORC_LIBRARY "Set to FALSE to use system ORC instead of bundled" ${NOT_UNBUNDLED}) - +if(ENABLE_ORC) include(cmake/find/snappy.cmake) +option(USE_INTERNAL_ORC_LIBRARY "Set to FALSE to use system ORC instead of bundled" ${NOT_UNBUNDLED}) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/orc/c++/include/orc/OrcFile.hh") if(USE_INTERNAL_ORC_LIBRARY) message(WARNING "submodule contrib/orc is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal ORC") set(USE_INTERNAL_ORC_LIBRARY 0) endif() set(MISSING_INTERNAL_ORC_LIBRARY 1) @@ -22,9 +14,6 @@ endif () if (NOT USE_INTERNAL_ORC_LIBRARY) find_package(orc) - if (NOT ORC_LIBRARY OR NOT ORC_INCLUDE_DIR) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system ORC") - endif () endif () #if (USE_INTERNAL_ORC_LIBRARY) @@ -41,10 +30,10 @@ elseif(NOT MISSING_INTERNAL_ORC_LIBRARY AND ARROW_LIBRARY AND SNAPPY_LIBRARY) # set(ORC_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/orc/c++/include") set(ORC_LIBRARY orc) set(USE_ORC 1) - set(USE_INTERNAL_ORC_LIBRARY 1) else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't enable ORC support - missing dependencies") set(USE_INTERNAL_ORC_LIBRARY 0) endif() +endif() + message (STATUS "Using internal=${USE_INTERNAL_ORC_LIBRARY} orc=${USE_ORC}: ${ORC_INCLUDE_DIR} : ${ORC_LIBRARY}") diff --git a/cmake/find/parquet.cmake b/cmake/find/parquet.cmake index 3cb6be38c3f..d4f62b87d29 100644 --- a/cmake/find/parquet.cmake +++ b/cmake/find/parquet.cmake @@ -1,39 +1,24 @@ if (Protobuf_PROTOC_EXECUTABLE) option (ENABLE_PARQUET "Enable parquet" ${ENABLE_LIBRARIES}) -elseif(ENABLE_PARQUET OR USE_INTERNAL_PARQUET_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use parquet without protoc executable") endif() -if (NOT ENABLE_PARQUET) - if(USE_INTERNAL_PARQUET_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal parquet with ENABLE_PARQUET=OFF") - endif() - message(STATUS "Building without Parquet support") - return() -endif() +if (ENABLE_PARQUET) if (NOT OS_FREEBSD) # Freebsd: ../contrib/arrow/cpp/src/arrow/util/bit-util.h:27:10: fatal error: endian.h: No such file or directory option(USE_INTERNAL_PARQUET_LIBRARY "Set to FALSE to use system parquet library instead of bundled" ${NOT_UNBUNDLED}) -elseif(USE_INTERNAL_PARQUET_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Using internal parquet is not supported on freebsd") endif() if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/CMakeLists.txt") if(USE_INTERNAL_PARQUET_LIBRARY) message(WARNING "submodule contrib/arrow (required for Parquet) is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal parquet library") - set(USE_INTERNAL_PARQUET_LIBRARY 0) endif() + set(USE_INTERNAL_PARQUET_LIBRARY 0) set(MISSING_INTERNAL_PARQUET_LIBRARY 1) endif() if(NOT USE_INTERNAL_PARQUET_LIBRARY) find_package(Arrow) find_package(Parquet) - - if(NOT ARROW_INCLUDE_DIR OR PARQUET_INCLUDE_DIR) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system parquet: arrow=${ARROW_INCLUDE_DIR} parquet=${PARQUET_INCLUDE_DIR}" ) - endif() endif() if(ARROW_INCLUDE_DIR AND PARQUET_INCLUDE_DIR) @@ -41,8 +26,6 @@ elseif(NOT MISSING_INTERNAL_PARQUET_LIBRARY AND NOT OS_FREEBSD) include(cmake/find/snappy.cmake) if(SNAPPY_LIBRARY) set(CAN_USE_INTERNAL_PARQUET_LIBRARY 1) - else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal parquet library without snappy") endif() include(CheckCXXSourceCompiles) @@ -50,20 +33,19 @@ elseif(NOT MISSING_INTERNAL_PARQUET_LIBRARY AND NOT OS_FREEBSD) set(CMAKE_REQUIRED_LIBRARIES ${DOUBLE_CONVERSION_LIBRARIES}) set(CMAKE_REQUIRED_INCLUDES ${DOUBLE_CONVERSION_INCLUDE_DIR}) check_cxx_source_compiles(" - #include <double-conversion/double-conversion.h> - int main() { static const int flags_ = double_conversion::StringToDoubleConverter::ALLOW_CASE_INSENSIBILITY; return 0;} + #include <double-conversion/double-conversion.h> + int main() { static const int flags_ = double_conversion::StringToDoubleConverter::ALLOW_CASE_INSENSIBILITY; return 0;} " HAVE_DOUBLE_CONVERSION_ALLOW_CASE_INSENSIBILITY) if(NOT HAVE_DOUBLE_CONVERSION_ALLOW_CASE_INSENSIBILITY) # HAVE_STD_RANDOM_SHUFFLE - message (${RECONFIGURE_MESSAGE_LEVEL} "Disabling internal parquet library because arrow is broken (can't use old double_conversion)") - set(CAN_USE_INTERNAL_PARQUET_LIBRARY 0) + message(STATUS "Disabling internal parquet library because arrow is broken (can't use old double_conversion)") + set(CAN_USE_INTERNAL_PARQUET_LIBRARY 0) endif() endif() - if(NOT CAN_USE_INTERNAL_PARQUET_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal parquet") + if(NOT CAN_USE_INTERNAL_PARQUET_LIBRARY) set(USE_INTERNAL_PARQUET_LIBRARY 0) - else() + else() set(USE_INTERNAL_PARQUET_LIBRARY 1) if(USE_INTERNAL_PARQUET_LIBRARY_NATIVE_CMAKE) @@ -71,7 +53,7 @@ elseif(NOT MISSING_INTERNAL_PARQUET_LIBRARY AND NOT OS_FREEBSD) set(PARQUET_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src" ${ClickHouse_BINARY_DIR}/contrib/arrow/cpp/src) endif() - if(MAKE_STATIC_LIBRARIES) + if(${USE_STATIC_LIBRARIES}) set(FLATBUFFERS_LIBRARY flatbuffers) set(ARROW_LIBRARY arrow_static) set(PARQUET_LIBRARY parquet_static) @@ -90,8 +72,8 @@ elseif(NOT MISSING_INTERNAL_PARQUET_LIBRARY AND NOT OS_FREEBSD) set(USE_ORC 1) set(USE_ARROW 1) endif() -elseif(OS_FREEBSD) - message (${RECONFIGURE_MESSAGE_LEVEL} "Using internal parquet library on FreeBSD is not supported") +endif() + endif() if(USE_PARQUET) diff --git a/cmake/find/protobuf.cmake b/cmake/find/protobuf.cmake index 49d09299f27..51110d3b6a3 100644 --- a/cmake/find/protobuf.cmake +++ b/cmake/find/protobuf.cmake @@ -1,18 +1,12 @@ option(ENABLE_PROTOBUF "Enable protobuf" ${ENABLE_LIBRARIES}) -if(NOT ENABLE_PROTOBUF) - if(USE_INTERNAL_PROTOBUF_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal protobuf with ENABLE_PROTOBUF=OFF") - endif() - return() -endif() +if(ENABLE_PROTOBUF) option(USE_INTERNAL_PROTOBUF_LIBRARY "Set to FALSE to use system protobuf instead of bundled" ${NOT_UNBUNDLED}) if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/protobuf/cmake/CMakeLists.txt") if(USE_INTERNAL_PROTOBUF_LIBRARY) message(WARNING "submodule contrib/protobuf is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal protobuf") set(USE_INTERNAL_PROTOBUF_LIBRARY 0) endif() set(MISSING_INTERNAL_PROTOBUF_LIBRARY 1) @@ -20,9 +14,6 @@ endif() if(NOT USE_INTERNAL_PROTOBUF_LIBRARY) find_package(Protobuf) - if (NOT Protobuf_LIBRARY OR NOT Protobuf_INCLUDE_DIR) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system protobuf") - endif() endif() if (Protobuf_LIBRARY AND Protobuf_INCLUDE_DIR) @@ -45,11 +36,11 @@ if(OS_FREEBSD AND SANITIZE STREQUAL "address") if(LLVM_INCLUDE_DIRS) set(Protobuf_INCLUDE_DIR ${Protobuf_INCLUDE_DIR} ${LLVM_INCLUDE_DIRS}) else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use protobuf on FreeBSD with address sanitizer without LLVM") set(USE_PROTOBUF 0) endif() endif() include (${ClickHouse_SOURCE_DIR}/cmake/protobuf_generate_cpp.cmake) +endif() message(STATUS "Using protobuf=${USE_PROTOBUF}: ${Protobuf_INCLUDE_DIR} : ${Protobuf_LIBRARY} : ${Protobuf_PROTOC_EXECUTABLE}") diff --git a/cmake/find/rapidjson.cmake b/cmake/find/rapidjson.cmake index f880d19143e..df896b644e5 100644 --- a/cmake/find/rapidjson.cmake +++ b/cmake/find/rapidjson.cmake @@ -1,8 +1,5 @@ option(ENABLE_RAPIDJSON "Use rapidjson" ${ENABLE_LIBRARIES}) if(NOT ENABLE_RAPIDJSON) - if(USE_INTERNAL_RAPIDJSON_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal rapidjson library with ENABLE_RAPIDJSON=OFF") - endif() return() endif() @@ -11,7 +8,6 @@ option(USE_INTERNAL_RAPIDJSON_LIBRARY "Set to FALSE to use system rapidjson libr if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rapidjson/include/rapidjson/rapidjson.h") if(USE_INTERNAL_RAPIDJSON_LIBRARY) message(WARNING "submodule contrib/rapidjson is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal rapidjson library") set(USE_INTERNAL_RAPIDJSON_LIBRARY 0) endif() set(MISSING_INTERNAL_RAPIDJSON_LIBRARY 1) @@ -19,9 +15,6 @@ endif() if(NOT USE_INTERNAL_RAPIDJSON_LIBRARY) find_path(RAPIDJSON_INCLUDE_DIR NAMES rapidjson/rapidjson.h PATHS ${RAPIDJSON_INCLUDE_PATHS}) - if(NOT RAPIDJSON_INCLUDE_DIR) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system rapidjson") - endif() endif() if(RAPIDJSON_INCLUDE_DIR) diff --git a/cmake/find/rdkafka.cmake b/cmake/find/rdkafka.cmake index 7b213fd37ea..f18674dd440 100644 --- a/cmake/find/rdkafka.cmake +++ b/cmake/find/rdkafka.cmake @@ -1,55 +1,30 @@ # Freebsd: contrib/cppkafka/include/cppkafka/detail/endianness.h:53:23: error: 'betoh16' was not declared in this scope if (NOT ARCH_ARM AND NOT OS_FREEBSD AND OPENSSL_FOUND) option (ENABLE_RDKAFKA "Enable kafka" ${ENABLE_LIBRARIES}) -elseif(ENABLE_RDKAFKA AND NOT OPENSSL_FOUND) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use librdkafka without SSL") -else(ENABLE_RDKAFKA) - message (${RECONFIGURE_MESSAGE_LEVEL} "librdafka is not supported on ARM and on FreeBSD") -endif () - -if (NOT ENABLE_RDKAFKA) - if (USE_INTERNAL_RDKAFKA_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal librdkafka with ENABLE_RDKAFKA=OFF") - endif() - return() -endif() - -if (NOT ARCH_ARM AND USE_LIBGSASL) - option (USE_INTERNAL_RDKAFKA_LIBRARY "Set to FALSE to use system librdkafka instead of the bundled" ${NOT_UNBUNDLED}) -elseif(USE_INTERNAL_RDKAFKA_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal librdkafka with ARCH_ARM=${ARCH_ARM} AND USE_LIBGSASL=${USE_LIBGSASL}") endif () if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cppkafka/CMakeLists.txt") - if(USE_INTERNAL_RDKAFKA_LIBRARY) - message (WARNING "submodule contrib/cppkafka is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal cppkafka") - set (USE_INTERNAL_RDKAFKA_LIBRARY 0) - endif() - set (MISSING_INTERNAL_CPPKAFKA_LIBRARY 1) + message (WARNING "submodule contrib/cppkafka is missing. to fix try run: \n git submodule update --init --recursive") + set (ENABLE_RDKAFKA 0) endif () -if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/librdkafka/CMakeLists.txt") - if(USE_INTERNAL_RDKAFKA_LIBRARY OR MISSING_INTERNAL_CPPKAFKA_LIBRARY) - message (WARNING "submodule contrib/librdkafka is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal rdkafka") - set (USE_INTERNAL_RDKAFKA_LIBRARY 0) - endif() - set (MISSING_INTERNAL_RDKAFKA_LIBRARY 1) +if (ENABLE_RDKAFKA) + +if (NOT ARCH_ARM AND USE_LIBGSASL) + option (USE_INTERNAL_RDKAFKA_LIBRARY "Set to FALSE to use system librdkafka instead of the bundled" ${NOT_UNBUNDLED}) +endif () + +if (USE_INTERNAL_RDKAFKA_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/librdkafka/CMakeLists.txt") + message (WARNING "submodule contrib/librdkafka is missing. to fix try run: \n git submodule update --init --recursive") + set (USE_INTERNAL_RDKAFKA_LIBRARY 0) + set (MISSING_INTERNAL_RDKAFKA_LIBRARY 1) endif () if (NOT USE_INTERNAL_RDKAFKA_LIBRARY) find_library (RDKAFKA_LIB rdkafka) find_path (RDKAFKA_INCLUDE_DIR NAMES librdkafka/rdkafka.h PATHS ${RDKAFKA_INCLUDE_PATHS}) - if (NOT RDKAFKA_LIB OR NOT RDKAFKA_INCLUDE_DIR) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system librdkafka") - endif() - if (USE_STATIC_LIBRARIES AND NOT OS_FREEBSD) find_library (SASL2_LIBRARY sasl2) - if (NOT SASL2_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system sasl2 library needed for static librdkafka") - endif() endif () set (CPPKAFKA_LIBRARY cppkafka) # TODO: try to use unbundled version. endif () @@ -64,14 +39,14 @@ if (RDKAFKA_LIB AND RDKAFKA_INCLUDE_DIR) if (LZ4_LIBRARY) list (APPEND RDKAFKA_LIBRARY ${LZ4_LIBRARY}) endif () -elseif (NOT MISSING_INTERNAL_RDKAFKA_LIBRARY AND NOT MISSING_INTERNAL_CPPKAFKA_LIBRARY AND NOT ARCH_ARM) +elseif (NOT MISSING_INTERNAL_RDKAFKA_LIBRARY AND NOT ARCH_ARM) set (USE_INTERNAL_RDKAFKA_LIBRARY 1) set (RDKAFKA_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/librdkafka/src") set (RDKAFKA_LIBRARY rdkafka) set (CPPKAFKA_LIBRARY cppkafka) set (USE_RDKAFKA 1) -elseif(ARCH_ARM) - message (${RECONFIGURE_MESSAGE_LEVEL} "Using internal rdkafka on ARM is not supported") +endif () + endif () message (STATUS "Using librdkafka=${USE_RDKAFKA}: ${RDKAFKA_INCLUDE_DIR} : ${RDKAFKA_LIBRARY} ${CPPKAFKA_LIBRARY}") diff --git a/cmake/find/re2.cmake b/cmake/find/re2.cmake index 87bc974c788..05ba80f143f 100644 --- a/cmake/find/re2.cmake +++ b/cmake/find/re2.cmake @@ -3,7 +3,6 @@ option (USE_INTERNAL_RE2_LIBRARY "Set to FALSE to use system re2 library instead if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/re2/CMakeLists.txt") if(USE_INTERNAL_RE2_LIBRARY) message(WARNING "submodule contrib/re2 is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal re2 library") endif() set(USE_INTERNAL_RE2_LIBRARY 0) set(MISSING_INTERNAL_RE2_LIBRARY 1) @@ -12,9 +11,6 @@ endif() if (NOT USE_INTERNAL_RE2_LIBRARY) find_library (RE2_LIBRARY re2) find_path (RE2_INCLUDE_DIR NAMES re2/re2.h PATHS ${RE2_INCLUDE_PATHS}) - if (NOT RE2_LIBRARY OR NOT RE2_INCLUDE_DIR) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system re2 library") - endif () endif () string(FIND ${CMAKE_CURRENT_BINARY_DIR} " " _have_space) @@ -28,13 +24,12 @@ if (RE2_LIBRARY AND RE2_INCLUDE_DIR) elseif (NOT MISSING_INTERNAL_RE2_LIBRARY) set (USE_INTERNAL_RE2_LIBRARY 1) set (RE2_LIBRARY re2) - set (RE2_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/re2") + set (RE2_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/re2) if (NOT MISSING_INTERNAL_RE2_ST_LIBRARY) set (RE2_ST_LIBRARY re2_st) set (USE_RE2_ST 1) else () set (RE2_ST_LIBRARY ${RE2_LIBRARY}) - message (${RECONFIGURE_MESSAGE_LEVEL} "Using internal re2 library instead of re2_st") endif () endif () diff --git a/cmake/find/s3.cmake b/cmake/find/s3.cmake index bf15c4944bb..af53dc80feb 100644 --- a/cmake/find/s3.cmake +++ b/cmake/find/s3.cmake @@ -1,39 +1,26 @@ if(NOT OS_FREEBSD AND NOT APPLE AND NOT ARCH_ARM) option(ENABLE_S3 "Enable S3" ${ENABLE_LIBRARIES}) -elseif(ENABLE_S3 OR USE_INTERNAL_AWS_S3_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use S3 on ARM, Apple or FreeBSD") endif() -if(NOT ENABLE_S3) - if(USE_INTERNAL_AWS_S3_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal S3 library with ENABLE_S3=OFF") - endif() - return() -endif() +if(ENABLE_S3) + option(USE_INTERNAL_AWS_S3_LIBRARY "Set to FALSE to use system S3 instead of bundled" ${NOT_UNBUNDLED}) -option(USE_INTERNAL_AWS_S3_LIBRARY "Set to FALSE to use system S3 instead of bundled" ${NOT_UNBUNDLED}) -if (NOT USE_INTERNAL_AWS_S3_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Compilation with external S3 library is not supported yet") -endif() - -if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-s3") - message (WARNING "submodule contrib/aws is missing. to fix try run: \n git submodule update --init --recursive") - if (USE_INTERNAL_AWS_S3_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal S3 library") + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-s3") + message (WARNING "submodule contrib/aws is missing. to fix try run: \n git submodule update --init --recursive") + set (MISSING_AWS_S3 1) endif () - set (MISSING_AWS_S3 1) -endif () -if (USE_INTERNAL_AWS_S3_LIBRARY AND NOT MISSING_AWS_S3) - set(AWS_S3_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-s3/include") - set(AWS_S3_CORE_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-core/include") - set(AWS_S3_LIBRARY aws_s3) - set(USE_INTERNAL_AWS_S3_LIBRARY 1) - set(USE_AWS_S3 1) -else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't enable S3") - set(USE_INTERNAL_AWS_S3_LIBRARY 0) - set(USE_AWS_S3 0) -endif () + if (USE_INTERNAL_AWS_S3_LIBRARY AND NOT MISSING_AWS_S3) + set(AWS_S3_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-s3/include") + set(AWS_S3_CORE_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-core/include") + set(AWS_S3_LIBRARY aws_s3) + set(USE_INTERNAL_AWS_S3_LIBRARY 1) + set(USE_AWS_S3 1) + else() + set(USE_INTERNAL_AWS_S3_LIBRARY 0) + set(USE_AWS_S3 0) + endif () + +endif() message (STATUS "Using aws_s3=${USE_AWS_S3}: ${AWS_S3_INCLUDE_DIR} : ${AWS_S3_LIBRARY}") diff --git a/cmake/find/sentry.cmake b/cmake/find/sentry.cmake index 2936c045f99..84425220f12 100644 --- a/cmake/find/sentry.cmake +++ b/cmake/find/sentry.cmake @@ -2,9 +2,6 @@ set (SENTRY_LIBRARY "sentry") set (SENTRY_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/sentry-native/include") if (NOT EXISTS "${SENTRY_INCLUDE_DIR}/sentry.h") message (WARNING "submodule contrib/sentry-native is missing. to fix try run: \n git submodule update --init --recursive") - if (USE_SENTRY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal sentry library") - endif() return() endif () @@ -19,6 +16,4 @@ if (NOT OS_FREEBSD AND NOT SPLIT_SHARED_LIBRARIES AND NOT_UNBUNDLED AND NOT (OS_ message (STATUS "Using sentry=${USE_SENTRY}: ${SENTRY_LIBRARY}") include_directories("${SENTRY_INCLUDE_DIR}") -elseif (USE_SENTRY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Sentry is not supported in current configuration") endif () diff --git a/cmake/find/simdjson.cmake b/cmake/find/simdjson.cmake index cffe20bdb2d..4b1f3224076 100644 --- a/cmake/find/simdjson.cmake +++ b/cmake/find/simdjson.cmake @@ -1,11 +1,8 @@ -option (USE_SIMDJSON "Use simdjson" ${ENABLE_LIBRARIES}) - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/simdjson/include/simdjson.h") message (WARNING "submodule contrib/simdjson is missing. to fix try run: \n git submodule update --init --recursive") - if (USE_SIMDJSON) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal simdjson library") - endif() return() endif () +option (USE_SIMDJSON "Use simdjson" ON) + message(STATUS "Using simdjson=${USE_SIMDJSON}") diff --git a/cmake/find/snappy.cmake b/cmake/find/snappy.cmake index e719231c338..504db87d613 100644 --- a/cmake/find/snappy.cmake +++ b/cmake/find/snappy.cmake @@ -1,21 +1,13 @@ option(USE_SNAPPY "Enable support of snappy library" ${ENABLE_LIBRARIES}) -if(NOT USE_SNAPPY) - if (USE_INTERNAL_SNAPPY_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal snappy library with USE_SNAPPY=OFF") +if(USE_SNAPPY) + option (USE_INTERNAL_SNAPPY_LIBRARY "Set to FALSE to use system snappy library instead of bundled" ${NOT_UNBUNDLED}) + + if(NOT USE_INTERNAL_SNAPPY_LIBRARY) + find_library(SNAPPY_LIBRARY snappy) + else () + set(SNAPPY_LIBRARY snappy) endif() - return() -endif() -option (USE_INTERNAL_SNAPPY_LIBRARY "Set to FALSE to use system snappy library instead of bundled" ${NOT_UNBUNDLED}) - -if(NOT USE_INTERNAL_SNAPPY_LIBRARY) - find_library(SNAPPY_LIBRARY snappy) - if (NOT SNAPPY_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system snappy library") - endif() -else () - set(SNAPPY_LIBRARY snappy) -endif() - -message (STATUS "Using snappy: ${SNAPPY_LIBRARY}") + message (STATUS "Using snappy: ${SNAPPY_LIBRARY}") +endif () diff --git a/cmake/find/sparsehash.cmake b/cmake/find/sparsehash.cmake index f8b6d293705..d34ed8e048b 100644 --- a/cmake/find/sparsehash.cmake +++ b/cmake/find/sparsehash.cmake @@ -2,9 +2,6 @@ option (USE_INTERNAL_SPARSEHASH_LIBRARY "Set to FALSE to use system sparsehash l if (NOT USE_INTERNAL_SPARSEHASH_LIBRARY) find_path (SPARSEHASH_INCLUDE_DIR NAMES sparsehash/sparse_hash_map PATHS ${SPARSEHASH_INCLUDE_PATHS}) - if (NOT SPARSEHASH_INCLUDE_DIR) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system sparsehash library") - endif () endif () if (SPARSEHASH_INCLUDE_DIR) diff --git a/cmake/find/ssl.cmake b/cmake/find/ssl.cmake index 9058857c173..efc9127309c 100644 --- a/cmake/find/ssl.cmake +++ b/cmake/find/ssl.cmake @@ -1,18 +1,12 @@ option(ENABLE_SSL "Enable ssl" ${ENABLE_LIBRARIES}) -if(NOT ENABLE_SSL) - if (USE_INTERNAL_SSL_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal ssl library with ENABLE_SSL=OFF") - endif() - return() -endif() +if(ENABLE_SSL) option(USE_INTERNAL_SSL_LIBRARY "Set to FALSE to use system *ssl library instead of bundled" ${NOT_UNBUNDLED}) if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/openssl/README") if(USE_INTERNAL_SSL_LIBRARY) message(WARNING "submodule contrib/openssl is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal ssl library") endif() set(USE_INTERNAL_SSL_LIBRARY 0) set(MISSING_INTERNAL_SSL_LIBRARY 1) @@ -42,10 +36,6 @@ if (NOT USE_INTERNAL_SSL_LIBRARY) set (OPENSSL_FOUND 1) endif () endif () - - if (NOT OPENSSL_FOUND) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system ssl") - endif() endif () if (NOT OPENSSL_FOUND AND NOT MISSING_INTERNAL_SSL_LIBRARY) @@ -133,4 +123,7 @@ if(OPENSSL_FOUND AND NOT USE_INTERNAL_SSL_LIBRARY) endif() endif() + +endif () + message (STATUS "Using ssl=${USE_SSL}: ${OPENSSL_INCLUDE_DIR} : ${OPENSSL_LIBRARIES}") diff --git a/cmake/find/stats.cmake b/cmake/find/stats.cmake index 339e8524598..ef5b1a73659 100644 --- a/cmake/find/stats.cmake +++ b/cmake/find/stats.cmake @@ -14,10 +14,6 @@ if (ENABLE_STATS) set(GCEM_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/gcem/include) set (USE_STATS 1) endif() - - if (NOT USE_STATS) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't enable stats library") - endif() endif() message (STATUS "Using stats=${USE_STATS} : ${STATS_INCLUDE_DIR}") diff --git a/cmake/find/termcap.cmake b/cmake/find/termcap.cmake index 80011cd825c..7564b7134e7 100644 --- a/cmake/find/termcap.cmake +++ b/cmake/find/termcap.cmake @@ -5,7 +5,4 @@ endif() if (NOT TERMCAP_LIBRARY) find_library (TERMCAP_LIBRARY termcap) endif() -if (NOT TERMCAP_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find tinfo, ncurses or termcap library") -endif() message (STATUS "Using termcap: ${TERMCAP_LIBRARY}") diff --git a/cmake/find/zlib.cmake b/cmake/find/zlib.cmake index 754cd8c85bb..f65d379f577 100644 --- a/cmake/find/zlib.cmake +++ b/cmake/find/zlib.cmake @@ -6,14 +6,12 @@ else () set (INTERNAL_ZLIB_NAME "zlib" CACHE INTERNAL "") if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}") message (WARNING "Will use standard zlib, please clone manually:\n git clone https://github.com/madler/zlib.git ${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal zlib library") endif () endif () if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}/zlib.h") if(USE_INTERNAL_ZLIB_LIBRARY) message(WARNING "submodule contrib/${INTERNAL_ZLIB_NAME} is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal zlib library") endif() set(USE_INTERNAL_ZLIB_LIBRARY 0) set(MISSING_INTERNAL_ZLIB_LIBRARY 1) @@ -21,7 +19,6 @@ endif() if (NOT USE_INTERNAL_ZLIB_LIBRARY) find_package (ZLIB) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system zlib library") endif () if (NOT ZLIB_FOUND AND NOT MISSING_INTERNAL_ZLIB_LIBRARY) diff --git a/cmake/find/zstd.cmake b/cmake/find/zstd.cmake index b12bb701e0e..e4f32d4b170 100644 --- a/cmake/find/zstd.cmake +++ b/cmake/find/zstd.cmake @@ -3,18 +3,14 @@ option (USE_INTERNAL_ZSTD_LIBRARY "Set to FALSE to use system zstd library inste if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/zstd/lib/zstd.h") if(USE_INTERNAL_ZSTD_LIBRARY) message(WARNING "submodule contrib/zstd is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal zstd library") - set(USE_INTERNAL_ZSTD_LIBRARY 0) endif() + set(USE_INTERNAL_ZSTD_LIBRARY 0) set(MISSING_INTERNAL_ZSTD_LIBRARY 1) endif() if (NOT USE_INTERNAL_ZSTD_LIBRARY) find_library (ZSTD_LIBRARY zstd) find_path (ZSTD_INCLUDE_DIR NAMES zstd.h PATHS ${ZSTD_INCLUDE_PATHS}) - if (NOT ZSTD_LIBRARY OR NOT ZSTD_INCLUDE_DIR) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system zstd library") - endif () endif () if (ZSTD_LIBRARY AND ZSTD_INCLUDE_DIR) diff --git a/cmake/tools.cmake b/cmake/tools.cmake index a6ea573a59d..95e00ad9951 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -41,23 +41,25 @@ STRING(REGEX MATCHALL "[0-9]+" COMPILER_VERSION_LIST ${CMAKE_CXX_COMPILER_VERSIO LIST(GET COMPILER_VERSION_LIST 0 COMPILER_VERSION_MAJOR) option (LINKER_NAME "Linker name or full path") -if (COMPILER_GCC AND NOT LINKER_NAME) +if (COMPILER_GCC) find_program (LLD_PATH NAMES "ld.lld") find_program (GOLD_PATH NAMES "ld.gold") -elseif (NOT LINKER_NAME) +else () find_program (LLD_PATH NAMES "ld.lld-${COMPILER_VERSION_MAJOR}" "lld-${COMPILER_VERSION_MAJOR}" "ld.lld" "lld") find_program (GOLD_PATH NAMES "ld.gold" "gold") endif () -if (OS_LINUX AND NOT LINKER_NAME) +if (OS_LINUX) # We prefer LLD linker over Gold or BFD on Linux. - if (LLD_PATH) - if (COMPILER_GCC) - # GCC driver requires one of supported linker names like "lld". - set (LINKER_NAME "lld") - else () - # Clang driver simply allows full linker path. - set (LINKER_NAME ${LLD_PATH}) + if (NOT LINKER_NAME) + if (LLD_PATH) + if (COMPILER_GCC) + # GCC driver requires one of supported linker names like "lld". + set (LINKER_NAME "lld") + else () + # Clang driver simply allows full linker path. + set (LINKER_NAME ${LLD_PATH}) + endif () endif () endif () diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 442f2534f6a..be8cb9a9708 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -80,7 +80,7 @@ set(FLATBUFFERS_BINARY_DIR ${ClickHouse_BINARY_DIR}/contrib/flatbuffers) set(FLATBUFFERS_INCLUDE_DIR ${FLATBUFFERS_SRC_DIR}/include) # set flatbuffers CMake options -if (MAKE_STATIC_LIBRARIES) +if (${USE_STATIC_LIBRARIES}) set(FLATBUFFERS_BUILD_FLATLIB ON CACHE BOOL "Enable the build of the flatbuffers library") set(FLATBUFFERS_BUILD_SHAREDLIB OFF CACHE BOOL "Disable the build of the flatbuffers shared library") else () diff --git a/contrib/boost-cmake/CMakeLists.txt b/contrib/boost-cmake/CMakeLists.txt index 4d6a2ff0e62..e92fe4b7159 100644 --- a/contrib/boost-cmake/CMakeLists.txt +++ b/contrib/boost-cmake/CMakeLists.txt @@ -1,52 +1,6 @@ option (USE_INTERNAL_BOOST_LIBRARY "Use internal Boost library" ${NOT_UNBUNDLED}) -if (NOT USE_INTERNAL_BOOST_LIBRARY) - # 1.70 like in contrib/boost - # 1.67 on CI - set(BOOST_VERSION 1.67) - - find_package(Boost ${BOOST_VERSION} COMPONENTS - system - filesystem - iostreams - program_options - regex - ) - - if(Boost_INCLUDE_DIR AND Boost_FILESYSTEM_LIBRARY AND Boost_FILESYSTEM_LIBRARY AND - Boost_PROGRAM_OPTIONS_LIBRARY AND Boost_REGEX_LIBRARY AND Boost_SYSTEM_LIBRARY) - - set(EXTERNAL_BOOST_FOUND 1) - - add_library (_boost_headers_only INTERFACE) - add_library (boost::headers_only ALIAS _boost_headers_only) - target_include_directories (_boost_headers_only SYSTEM BEFORE INTERFACE ${Boost_INCLUDE_DIR}) - - add_library (_boost_filesystem INTERFACE) - add_library (_boost_iostreams INTERFACE) - add_library (_boost_program_options INTERFACE) - add_library (_boost_regex INTERFACE) - add_library (_boost_system INTERFACE) - - target_link_libraries (_boost_filesystem INTERFACE ${Boost_FILESYSTEM_LIBRARY}) - target_link_libraries (_boost_iostreams INTERFACE ${Boost_IOSTREAMS_LIBRARY}) - target_link_libraries (_boost_program_options INTERFACE ${Boost_PROGRAM_OPTIONS_LIBRARY}) - target_link_libraries (_boost_regex INTERFACE ${Boost_REGEX_LIBRARY}) - target_link_libraries (_boost_system INTERFACE ${Boost_SYSTEM_LIBRARY}) - - add_library (boost::filesystem ALIAS _boost_filesystem) - add_library (boost::iostreams ALIAS _boost_iostreams) - add_library (boost::program_options ALIAS _boost_program_options) - add_library (boost::regex ALIAS _boost_regex) - add_library (boost::system ALIAS _boost_system) - else() - set(EXTERNAL_BOOST_FOUND 0) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system boost") - endif() -endif() - -if (NOT EXTERNAL_BOOST_FOUND) - set (USE_INTERNAL_BOOST_LIBRARY 1) +if (USE_INTERNAL_BOOST_LIBRARY) set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/boost) # filesystem @@ -142,4 +96,38 @@ if (NOT EXTERNAL_BOOST_FOUND) add_library (_boost_system ${SRCS_SYSTEM}) add_library (boost::system ALIAS _boost_system) target_include_directories (_boost_system PRIVATE ${LIBRARY_DIR}) +else () + # 1.70 like in contrib/boost + # 1.67 on CI + set(BOOST_VERSION 1.67) + + find_package(Boost ${BOOST_VERSION} COMPONENTS + system + filesystem + iostreams + program_options + regex + REQUIRED) + + add_library (_boost_headers_only INTERFACE) + add_library (boost::headers_only ALIAS _boost_headers_only) + target_include_directories (_boost_headers_only SYSTEM BEFORE INTERFACE ${Boost_INCLUDE_DIR}) + + add_library (_boost_filesystem INTERFACE) + add_library (_boost_iostreams INTERFACE) + add_library (_boost_program_options INTERFACE) + add_library (_boost_regex INTERFACE) + add_library (_boost_system INTERFACE) + + target_link_libraries (_boost_filesystem INTERFACE ${Boost_FILESYSTEM_LIBRARY}) + target_link_libraries (_boost_iostreams INTERFACE ${Boost_IOSTREAMS_LIBRARY}) + target_link_libraries (_boost_program_options INTERFACE ${Boost_PROGRAM_OPTIONS_LIBRARY}) + target_link_libraries (_boost_regex INTERFACE ${Boost_REGEX_LIBRARY}) + target_link_libraries (_boost_system INTERFACE ${Boost_SYSTEM_LIBRARY}) + + add_library (boost::filesystem ALIAS _boost_filesystem) + add_library (boost::iostreams ALIAS _boost_iostreams) + add_library (boost::program_options ALIAS _boost_program_options) + add_library (boost::regex ALIAS _boost_regex) + add_library (boost::system ALIAS _boost_system) endif () diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index a4c15ed0b5c..ee87b3233ad 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -1,42 +1,9 @@ -option (USE_INTERNAL_CCTZ_LIBRARY "Use internal cctz library" ${NOT_UNBUNDLED}) +option (USE_INTERNAL_CCTZ "Use internal cctz library" ${NOT_UNBUNDLED}) -if (NOT USE_INTERNAL_CCTZ_LIBRARY) - find_library (LIBRARY_CCTZ cctz) - find_path (INCLUDE_CCTZ NAMES cctz/civil_time.h) +if (USE_INTERNAL_CCTZ) + SET(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/cctz) - if (LIBRARY_CCTZ AND INCLUDE_CCTZ) - set (EXTERNAL_CCTZ_LIBRARY_FOUND 1) - - set(CMAKE_REQUIRED_LIBRARIES ${LIBRARY_CCTZ}) - set(CMAKE_REQUIRED_INCLUDES ${INCLUDE_CCTZ}) - check_cxx_source_compiles( - " - #include <cctz/civil_time.h> - int main() { - cctz::civil_day date; - } - " - EXTERNAL_CCTZ_LIBRARY_WORKS - ) - - if (NOT EXTERNAL_CCTZ_LIBRARY_WORKS) - message (${RECONFIGURE_MESSAGE_LEVEL} "External cctz is not working: ${LIBRARY_CCTZ} ${INCLUDE_CCTZ}") - else() - add_library (cctz UNKNOWN IMPORTED) - set_property (TARGET cctz PROPERTY IMPORTED_LOCATION ${LIBRARY_CCTZ}) - set_property (TARGET cctz PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_CCTZ}) - endif() - else() - set (EXTERNAL_CCTZ_LIBRARY_FOUND 0) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system cctz") - endif() -endif() - -if (NOT EXTERNAL_CCTZ_LIBRARY_FOUND OR NOT EXTERNAL_CCTZ_LIBRARY_WORKS) - set(USE_INTERNAL_CCTZ_LIBRARY 1) - set(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/cctz) - - set (SRCS + SET (SRCS ${LIBRARY_DIR}/src/civil_time_detail.cc ${LIBRARY_DIR}/src/time_zone_fixed.cc ${LIBRARY_DIR}/src/time_zone_format.cc @@ -651,6 +618,29 @@ if (NOT EXTERNAL_CCTZ_LIBRARY_FOUND OR NOT EXTERNAL_CCTZ_LIBRARY_WORKS) add_dependencies(cctz tzdata) target_link_libraries(cctz INTERFACE "-Wl,${WHOLE_ARCHIVE} $<TARGET_FILE:tzdata> -Wl,${NO_WHOLE_ARCHIVE}") endif () + +else () + find_library (LIBRARY_CCTZ cctz) + find_path (INCLUDE_CCTZ NAMES cctz/civil_time.h) + + add_library (cctz UNKNOWN IMPORTED) + set_property (TARGET cctz PROPERTY IMPORTED_LOCATION ${LIBRARY_CCTZ}) + set_property (TARGET cctz PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_CCTZ}) + + set(CMAKE_REQUIRED_LIBRARIES cctz) + check_cxx_source_compiles( + " + #include <cctz/civil_time.h> + int main() { + cctz::civil_day date; + } + " + EXTERNAL_CCTZ_WORKS + ) + + if (NOT EXTERNAL_CCTZ_WORKS) + message (FATAL_ERROR "cctz is unusable: ${LIBRARY_CCTZ} ${INCLUDE_CCTZ}") + endif () endif () message (STATUS "Using cctz") diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index eb97cfa5f14..3c3226cae9e 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -1,200 +1,187 @@ option (ENABLE_CURL "Enable curl" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_CURL) +if (ENABLE_CURL) + option (USE_INTERNAL_CURL "Use internal curl library" ${NOT_UNBUNDLED}) + if (USE_INTERNAL_CURL) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal curl with ENABLE_CURL=OFF") - endif() - return() -endif() + set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/curl") -option (USE_INTERNAL_CURL "Use internal curl library" ${NOT_UNBUNDLED}) + set (SRCS + ${LIBRARY_DIR}/lib/file.c + ${LIBRARY_DIR}/lib/timeval.c + ${LIBRARY_DIR}/lib/base64.c + ${LIBRARY_DIR}/lib/hostip.c + ${LIBRARY_DIR}/lib/progress.c + ${LIBRARY_DIR}/lib/formdata.c + ${LIBRARY_DIR}/lib/cookie.c + ${LIBRARY_DIR}/lib/http.c + ${LIBRARY_DIR}/lib/sendf.c + ${LIBRARY_DIR}/lib/url.c + ${LIBRARY_DIR}/lib/dict.c + ${LIBRARY_DIR}/lib/if2ip.c + ${LIBRARY_DIR}/lib/speedcheck.c + ${LIBRARY_DIR}/lib/ldap.c + ${LIBRARY_DIR}/lib/version.c + ${LIBRARY_DIR}/lib/getenv.c + ${LIBRARY_DIR}/lib/escape.c + ${LIBRARY_DIR}/lib/mprintf.c + ${LIBRARY_DIR}/lib/telnet.c + ${LIBRARY_DIR}/lib/netrc.c + ${LIBRARY_DIR}/lib/getinfo.c + ${LIBRARY_DIR}/lib/transfer.c + ${LIBRARY_DIR}/lib/strcase.c + ${LIBRARY_DIR}/lib/easy.c + ${LIBRARY_DIR}/lib/security.c + ${LIBRARY_DIR}/lib/curl_fnmatch.c + ${LIBRARY_DIR}/lib/fileinfo.c + ${LIBRARY_DIR}/lib/wildcard.c + ${LIBRARY_DIR}/lib/krb5.c + ${LIBRARY_DIR}/lib/memdebug.c + ${LIBRARY_DIR}/lib/http_chunks.c + ${LIBRARY_DIR}/lib/strtok.c + ${LIBRARY_DIR}/lib/connect.c + ${LIBRARY_DIR}/lib/llist.c + ${LIBRARY_DIR}/lib/hash.c + ${LIBRARY_DIR}/lib/multi.c + ${LIBRARY_DIR}/lib/content_encoding.c + ${LIBRARY_DIR}/lib/share.c + ${LIBRARY_DIR}/lib/http_digest.c + ${LIBRARY_DIR}/lib/md4.c + ${LIBRARY_DIR}/lib/md5.c + ${LIBRARY_DIR}/lib/http_negotiate.c + ${LIBRARY_DIR}/lib/inet_pton.c + ${LIBRARY_DIR}/lib/strtoofft.c + ${LIBRARY_DIR}/lib/strerror.c + ${LIBRARY_DIR}/lib/amigaos.c + ${LIBRARY_DIR}/lib/hostasyn.c + ${LIBRARY_DIR}/lib/hostip4.c + ${LIBRARY_DIR}/lib/hostip6.c + ${LIBRARY_DIR}/lib/hostsyn.c + ${LIBRARY_DIR}/lib/inet_ntop.c + ${LIBRARY_DIR}/lib/parsedate.c + ${LIBRARY_DIR}/lib/select.c + ${LIBRARY_DIR}/lib/splay.c + ${LIBRARY_DIR}/lib/strdup.c + ${LIBRARY_DIR}/lib/socks.c + ${LIBRARY_DIR}/lib/curl_addrinfo.c + ${LIBRARY_DIR}/lib/socks_gssapi.c + ${LIBRARY_DIR}/lib/socks_sspi.c + ${LIBRARY_DIR}/lib/curl_sspi.c + ${LIBRARY_DIR}/lib/slist.c + ${LIBRARY_DIR}/lib/nonblock.c + ${LIBRARY_DIR}/lib/curl_memrchr.c + ${LIBRARY_DIR}/lib/imap.c + ${LIBRARY_DIR}/lib/pop3.c + ${LIBRARY_DIR}/lib/smtp.c + ${LIBRARY_DIR}/lib/pingpong.c + ${LIBRARY_DIR}/lib/rtsp.c + ${LIBRARY_DIR}/lib/curl_threads.c + ${LIBRARY_DIR}/lib/warnless.c + ${LIBRARY_DIR}/lib/hmac.c + ${LIBRARY_DIR}/lib/curl_rtmp.c + ${LIBRARY_DIR}/lib/openldap.c + ${LIBRARY_DIR}/lib/curl_gethostname.c + ${LIBRARY_DIR}/lib/gopher.c + ${LIBRARY_DIR}/lib/idn_win32.c + ${LIBRARY_DIR}/lib/http_proxy.c + ${LIBRARY_DIR}/lib/non-ascii.c + ${LIBRARY_DIR}/lib/asyn-thread.c + ${LIBRARY_DIR}/lib/curl_gssapi.c + ${LIBRARY_DIR}/lib/http_ntlm.c + ${LIBRARY_DIR}/lib/curl_ntlm_wb.c + ${LIBRARY_DIR}/lib/curl_ntlm_core.c + ${LIBRARY_DIR}/lib/curl_sasl.c + ${LIBRARY_DIR}/lib/rand.c + ${LIBRARY_DIR}/lib/curl_multibyte.c + ${LIBRARY_DIR}/lib/hostcheck.c + ${LIBRARY_DIR}/lib/conncache.c + ${LIBRARY_DIR}/lib/dotdot.c + ${LIBRARY_DIR}/lib/x509asn1.c + ${LIBRARY_DIR}/lib/http2.c + ${LIBRARY_DIR}/lib/smb.c + ${LIBRARY_DIR}/lib/curl_endian.c + ${LIBRARY_DIR}/lib/curl_des.c + ${LIBRARY_DIR}/lib/system_win32.c + ${LIBRARY_DIR}/lib/mime.c + ${LIBRARY_DIR}/lib/sha256.c + ${LIBRARY_DIR}/lib/setopt.c + ${LIBRARY_DIR}/lib/curl_path.c + ${LIBRARY_DIR}/lib/curl_ctype.c + ${LIBRARY_DIR}/lib/curl_range.c + ${LIBRARY_DIR}/lib/psl.c + ${LIBRARY_DIR}/lib/doh.c + ${LIBRARY_DIR}/lib/urlapi.c + ${LIBRARY_DIR}/lib/curl_get_line.c + ${LIBRARY_DIR}/lib/altsvc.c + ${LIBRARY_DIR}/lib/socketpair.c + ${LIBRARY_DIR}/lib/vauth/vauth.c + ${LIBRARY_DIR}/lib/vauth/cleartext.c + ${LIBRARY_DIR}/lib/vauth/cram.c + ${LIBRARY_DIR}/lib/vauth/digest.c + ${LIBRARY_DIR}/lib/vauth/digest_sspi.c + ${LIBRARY_DIR}/lib/vauth/krb5_gssapi.c + ${LIBRARY_DIR}/lib/vauth/krb5_sspi.c + ${LIBRARY_DIR}/lib/vauth/ntlm.c + ${LIBRARY_DIR}/lib/vauth/ntlm_sspi.c + ${LIBRARY_DIR}/lib/vauth/oauth2.c + ${LIBRARY_DIR}/lib/vauth/spnego_gssapi.c + ${LIBRARY_DIR}/lib/vauth/spnego_sspi.c + ${LIBRARY_DIR}/lib/vtls/openssl.c + ${LIBRARY_DIR}/lib/vtls/gtls.c + ${LIBRARY_DIR}/lib/vtls/vtls.c + ${LIBRARY_DIR}/lib/vtls/nss.c + ${LIBRARY_DIR}/lib/vtls/polarssl.c + ${LIBRARY_DIR}/lib/vtls/polarssl_threadlock.c + ${LIBRARY_DIR}/lib/vtls/wolfssl.c + ${LIBRARY_DIR}/lib/vtls/schannel.c + ${LIBRARY_DIR}/lib/vtls/schannel_verify.c + ${LIBRARY_DIR}/lib/vtls/sectransp.c + ${LIBRARY_DIR}/lib/vtls/gskit.c + ${LIBRARY_DIR}/lib/vtls/mbedtls.c + ${LIBRARY_DIR}/lib/vtls/mesalink.c + ${LIBRARY_DIR}/lib/vtls/bearssl.c + ${LIBRARY_DIR}/lib/vquic/ngtcp2.c + ${LIBRARY_DIR}/lib/vquic/quiche.c + ${LIBRARY_DIR}/lib/vssh/libssh2.c + ${LIBRARY_DIR}/lib/vssh/libssh.c + ) -if (NOT USE_INTERNAL_CURL) - find_package (CURL) - if (NOT CURL_FOUND) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system curl") - endif() -endif() + add_library (curl ${SRCS}) -if (NOT CURL_FOUND) - set (USE_INTERNAL_CURL 1) - set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/curl") + target_compile_definitions (curl PRIVATE + HAVE_CONFIG_H + BUILDING_LIBCURL + CURL_HIDDEN_SYMBOLS + libcurl_EXPORTS + OS="${CMAKE_SYSTEM_NAME}" + ) + target_include_directories (curl PUBLIC + ${LIBRARY_DIR}/include + ${LIBRARY_DIR}/lib + . # curl_config.h + ) - set (SRCS - ${LIBRARY_DIR}/lib/file.c - ${LIBRARY_DIR}/lib/timeval.c - ${LIBRARY_DIR}/lib/base64.c - ${LIBRARY_DIR}/lib/hostip.c - ${LIBRARY_DIR}/lib/progress.c - ${LIBRARY_DIR}/lib/formdata.c - ${LIBRARY_DIR}/lib/cookie.c - ${LIBRARY_DIR}/lib/http.c - ${LIBRARY_DIR}/lib/sendf.c - ${LIBRARY_DIR}/lib/url.c - ${LIBRARY_DIR}/lib/dict.c - ${LIBRARY_DIR}/lib/if2ip.c - ${LIBRARY_DIR}/lib/speedcheck.c - ${LIBRARY_DIR}/lib/ldap.c - ${LIBRARY_DIR}/lib/version.c - ${LIBRARY_DIR}/lib/getenv.c - ${LIBRARY_DIR}/lib/escape.c - ${LIBRARY_DIR}/lib/mprintf.c - ${LIBRARY_DIR}/lib/telnet.c - ${LIBRARY_DIR}/lib/netrc.c - ${LIBRARY_DIR}/lib/getinfo.c - ${LIBRARY_DIR}/lib/transfer.c - ${LIBRARY_DIR}/lib/strcase.c - ${LIBRARY_DIR}/lib/easy.c - ${LIBRARY_DIR}/lib/security.c - ${LIBRARY_DIR}/lib/curl_fnmatch.c - ${LIBRARY_DIR}/lib/fileinfo.c - ${LIBRARY_DIR}/lib/wildcard.c - ${LIBRARY_DIR}/lib/krb5.c - ${LIBRARY_DIR}/lib/memdebug.c - ${LIBRARY_DIR}/lib/http_chunks.c - ${LIBRARY_DIR}/lib/strtok.c - ${LIBRARY_DIR}/lib/connect.c - ${LIBRARY_DIR}/lib/llist.c - ${LIBRARY_DIR}/lib/hash.c - ${LIBRARY_DIR}/lib/multi.c - ${LIBRARY_DIR}/lib/content_encoding.c - ${LIBRARY_DIR}/lib/share.c - ${LIBRARY_DIR}/lib/http_digest.c - ${LIBRARY_DIR}/lib/md4.c - ${LIBRARY_DIR}/lib/md5.c - ${LIBRARY_DIR}/lib/http_negotiate.c - ${LIBRARY_DIR}/lib/inet_pton.c - ${LIBRARY_DIR}/lib/strtoofft.c - ${LIBRARY_DIR}/lib/strerror.c - ${LIBRARY_DIR}/lib/amigaos.c - ${LIBRARY_DIR}/lib/hostasyn.c - ${LIBRARY_DIR}/lib/hostip4.c - ${LIBRARY_DIR}/lib/hostip6.c - ${LIBRARY_DIR}/lib/hostsyn.c - ${LIBRARY_DIR}/lib/inet_ntop.c - ${LIBRARY_DIR}/lib/parsedate.c - ${LIBRARY_DIR}/lib/select.c - ${LIBRARY_DIR}/lib/splay.c - ${LIBRARY_DIR}/lib/strdup.c - ${LIBRARY_DIR}/lib/socks.c - ${LIBRARY_DIR}/lib/curl_addrinfo.c - ${LIBRARY_DIR}/lib/socks_gssapi.c - ${LIBRARY_DIR}/lib/socks_sspi.c - ${LIBRARY_DIR}/lib/curl_sspi.c - ${LIBRARY_DIR}/lib/slist.c - ${LIBRARY_DIR}/lib/nonblock.c - ${LIBRARY_DIR}/lib/curl_memrchr.c - ${LIBRARY_DIR}/lib/imap.c - ${LIBRARY_DIR}/lib/pop3.c - ${LIBRARY_DIR}/lib/smtp.c - ${LIBRARY_DIR}/lib/pingpong.c - ${LIBRARY_DIR}/lib/rtsp.c - ${LIBRARY_DIR}/lib/curl_threads.c - ${LIBRARY_DIR}/lib/warnless.c - ${LIBRARY_DIR}/lib/hmac.c - ${LIBRARY_DIR}/lib/curl_rtmp.c - ${LIBRARY_DIR}/lib/openldap.c - ${LIBRARY_DIR}/lib/curl_gethostname.c - ${LIBRARY_DIR}/lib/gopher.c - ${LIBRARY_DIR}/lib/idn_win32.c - ${LIBRARY_DIR}/lib/http_proxy.c - ${LIBRARY_DIR}/lib/non-ascii.c - ${LIBRARY_DIR}/lib/asyn-thread.c - ${LIBRARY_DIR}/lib/curl_gssapi.c - ${LIBRARY_DIR}/lib/http_ntlm.c - ${LIBRARY_DIR}/lib/curl_ntlm_wb.c - ${LIBRARY_DIR}/lib/curl_ntlm_core.c - ${LIBRARY_DIR}/lib/curl_sasl.c - ${LIBRARY_DIR}/lib/rand.c - ${LIBRARY_DIR}/lib/curl_multibyte.c - ${LIBRARY_DIR}/lib/hostcheck.c - ${LIBRARY_DIR}/lib/conncache.c - ${LIBRARY_DIR}/lib/dotdot.c - ${LIBRARY_DIR}/lib/x509asn1.c - ${LIBRARY_DIR}/lib/http2.c - ${LIBRARY_DIR}/lib/smb.c - ${LIBRARY_DIR}/lib/curl_endian.c - ${LIBRARY_DIR}/lib/curl_des.c - ${LIBRARY_DIR}/lib/system_win32.c - ${LIBRARY_DIR}/lib/mime.c - ${LIBRARY_DIR}/lib/sha256.c - ${LIBRARY_DIR}/lib/setopt.c - ${LIBRARY_DIR}/lib/curl_path.c - ${LIBRARY_DIR}/lib/curl_ctype.c - ${LIBRARY_DIR}/lib/curl_range.c - ${LIBRARY_DIR}/lib/psl.c - ${LIBRARY_DIR}/lib/doh.c - ${LIBRARY_DIR}/lib/urlapi.c - ${LIBRARY_DIR}/lib/curl_get_line.c - ${LIBRARY_DIR}/lib/altsvc.c - ${LIBRARY_DIR}/lib/socketpair.c - ${LIBRARY_DIR}/lib/vauth/vauth.c - ${LIBRARY_DIR}/lib/vauth/cleartext.c - ${LIBRARY_DIR}/lib/vauth/cram.c - ${LIBRARY_DIR}/lib/vauth/digest.c - ${LIBRARY_DIR}/lib/vauth/digest_sspi.c - ${LIBRARY_DIR}/lib/vauth/krb5_gssapi.c - ${LIBRARY_DIR}/lib/vauth/krb5_sspi.c - ${LIBRARY_DIR}/lib/vauth/ntlm.c - ${LIBRARY_DIR}/lib/vauth/ntlm_sspi.c - ${LIBRARY_DIR}/lib/vauth/oauth2.c - ${LIBRARY_DIR}/lib/vauth/spnego_gssapi.c - ${LIBRARY_DIR}/lib/vauth/spnego_sspi.c - ${LIBRARY_DIR}/lib/vtls/openssl.c - ${LIBRARY_DIR}/lib/vtls/gtls.c - ${LIBRARY_DIR}/lib/vtls/vtls.c - ${LIBRARY_DIR}/lib/vtls/nss.c - ${LIBRARY_DIR}/lib/vtls/polarssl.c - ${LIBRARY_DIR}/lib/vtls/polarssl_threadlock.c - ${LIBRARY_DIR}/lib/vtls/wolfssl.c - ${LIBRARY_DIR}/lib/vtls/schannel.c - ${LIBRARY_DIR}/lib/vtls/schannel_verify.c - ${LIBRARY_DIR}/lib/vtls/sectransp.c - ${LIBRARY_DIR}/lib/vtls/gskit.c - ${LIBRARY_DIR}/lib/vtls/mbedtls.c - ${LIBRARY_DIR}/lib/vtls/mesalink.c - ${LIBRARY_DIR}/lib/vtls/bearssl.c - ${LIBRARY_DIR}/lib/vquic/ngtcp2.c - ${LIBRARY_DIR}/lib/vquic/quiche.c - ${LIBRARY_DIR}/lib/vssh/libssh2.c - ${LIBRARY_DIR}/lib/vssh/libssh.c - ) + target_link_libraries (curl PRIVATE ssl) - add_library (curl ${SRCS}) + # The library is large - avoid bloat (XXX: is it?) + target_compile_options (curl PRIVATE -g0) - target_compile_definitions (curl PRIVATE - HAVE_CONFIG_H - BUILDING_LIBCURL - CURL_HIDDEN_SYMBOLS - libcurl_EXPORTS - OS="${CMAKE_SYSTEM_NAME}" - ) - target_include_directories (curl PUBLIC - ${LIBRARY_DIR}/include - ${LIBRARY_DIR}/lib - . # curl_config.h - ) - - target_link_libraries (curl PRIVATE ssl) - - # The library is large - avoid bloat (XXX: is it?) - target_compile_options (curl PRIVATE -g0) - - # find_package(CURL) compatibility for the following packages that uses - # find_package(CURL)/include(FindCURL): - # - mariadb-connector-c - # - aws-s3-cmake - # - sentry-native - set (CURL_FOUND ON CACHE BOOL "") - set (CURL_ROOT_DIR ${LIBRARY_DIR} CACHE PATH "") - set (CURL_INCLUDE_DIR ${LIBRARY_DIR}/include CACHE PATH "") - set (CURL_INCLUDE_DIRS ${LIBRARY_DIR}/include CACHE PATH "") - set (CURL_LIBRARY curl CACHE STRING "") - set (CURL_LIBRARIES ${CURL_LIBRARY} CACHE STRING "") - set (CURL_VERSION_STRING 7.67.0 CACHE STRING "") - add_library (CURL::libcurl ALIAS ${CURL_LIBRARY}) - - set (USE_INTERNAL_CURL 1) + # find_package(CURL) compatibility for the following packages that uses + # find_package(CURL)/include(FindCURL): + # - mariadb-connector-c + # - aws-s3-cmake + # - sentry-native + set (CURL_FOUND ON CACHE BOOL "") + set (CURL_ROOT_DIR ${LIBRARY_DIR} CACHE PATH "") + set (CURL_INCLUDE_DIR ${LIBRARY_DIR}/include CACHE PATH "") + set (CURL_INCLUDE_DIRS ${LIBRARY_DIR}/include CACHE PATH "") + set (CURL_LIBRARY curl CACHE STRING "") + set (CURL_LIBRARIES ${CURL_LIBRARY} CACHE STRING "") + set (CURL_VERSION_STRING 7.67.0 CACHE STRING "") + add_library (CURL::libcurl ALIAS ${CURL_LIBRARY}) + else () + find_package (CURL REQUIRED) + endif () endif () message (STATUS "Using curl: ${CURL_INCLUDE_DIRS} : ${CURL_LIBRARIES}") diff --git a/contrib/hyperscan-cmake/CMakeLists.txt b/contrib/hyperscan-cmake/CMakeLists.txt index e403eeb9d3e..bc2f6c4e89d 100644 --- a/contrib/hyperscan-cmake/CMakeLists.txt +++ b/contrib/hyperscan-cmake/CMakeLists.txt @@ -1,272 +1,252 @@ -if (HAVE_SSSE3) - option (ENABLE_HYPERSCAN "Enable hyperscan library" ${ENABLE_LIBRARIES}) -elseif(ENABLE_HYPERSCAN) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use hyperscan without SSSE3") +option (ENABLE_HYPERSCAN "Enable hyperscan library" ${ENABLE_LIBRARIES}) + +if (NOT HAVE_SSSE3) set (ENABLE_HYPERSCAN OFF) endif () -if (NOT ENABLE_HYPERSCAN) +if (ENABLE_HYPERSCAN) + option (USE_INTERNAL_HYPERSCAN_LIBRARY "Use internal hyperscan library" ${NOT_UNBUNDLED}) + if (USE_INTERNAL_HYPERSCAN_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal hyperscan with ENABLE_HYPERSCAN=OFF") - endif() + set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/hyperscan) - add_library (hyperscan INTERFACE) - target_compile_definitions (hyperscan INTERFACE USE_HYPERSCAN=0) + set (SRCS + ${LIBRARY_DIR}/src/alloc.c + ${LIBRARY_DIR}/src/compiler/asserts.cpp + ${LIBRARY_DIR}/src/compiler/compiler.cpp + ${LIBRARY_DIR}/src/compiler/error.cpp + ${LIBRARY_DIR}/src/crc32.c + ${LIBRARY_DIR}/src/database.c + ${LIBRARY_DIR}/src/fdr/engine_description.cpp + ${LIBRARY_DIR}/src/fdr/fdr_compile_util.cpp + ${LIBRARY_DIR}/src/fdr/fdr_compile.cpp + ${LIBRARY_DIR}/src/fdr/fdr_confirm_compile.cpp + ${LIBRARY_DIR}/src/fdr/fdr_engine_description.cpp + ${LIBRARY_DIR}/src/fdr/fdr.c + ${LIBRARY_DIR}/src/fdr/flood_compile.cpp + ${LIBRARY_DIR}/src/fdr/teddy_compile.cpp + ${LIBRARY_DIR}/src/fdr/teddy_engine_description.cpp + ${LIBRARY_DIR}/src/fdr/teddy.c + ${LIBRARY_DIR}/src/grey.cpp + ${LIBRARY_DIR}/src/hs_valid_platform.c + ${LIBRARY_DIR}/src/hs_version.c + ${LIBRARY_DIR}/src/hs.cpp + ${LIBRARY_DIR}/src/hwlm/hwlm_build.cpp + ${LIBRARY_DIR}/src/hwlm/hwlm_literal.cpp + ${LIBRARY_DIR}/src/hwlm/hwlm.c + ${LIBRARY_DIR}/src/hwlm/noodle_build.cpp + ${LIBRARY_DIR}/src/hwlm/noodle_engine.c + ${LIBRARY_DIR}/src/nfa/accel_dfa_build_strat.cpp + ${LIBRARY_DIR}/src/nfa/accel.c + ${LIBRARY_DIR}/src/nfa/accelcompile.cpp + ${LIBRARY_DIR}/src/nfa/castle.c + ${LIBRARY_DIR}/src/nfa/castlecompile.cpp + ${LIBRARY_DIR}/src/nfa/dfa_build_strat.cpp + ${LIBRARY_DIR}/src/nfa/dfa_min.cpp + ${LIBRARY_DIR}/src/nfa/gough.c + ${LIBRARY_DIR}/src/nfa/goughcompile_accel.cpp + ${LIBRARY_DIR}/src/nfa/goughcompile_reg.cpp + ${LIBRARY_DIR}/src/nfa/goughcompile.cpp + ${LIBRARY_DIR}/src/nfa/lbr.c + ${LIBRARY_DIR}/src/nfa/limex_64.c + ${LIBRARY_DIR}/src/nfa/limex_accel.c + ${LIBRARY_DIR}/src/nfa/limex_compile.cpp + ${LIBRARY_DIR}/src/nfa/limex_native.c + ${LIBRARY_DIR}/src/nfa/limex_simd128.c + ${LIBRARY_DIR}/src/nfa/limex_simd256.c + ${LIBRARY_DIR}/src/nfa/limex_simd384.c + ${LIBRARY_DIR}/src/nfa/limex_simd512.c + ${LIBRARY_DIR}/src/nfa/mcclellan.c + ${LIBRARY_DIR}/src/nfa/mcclellancompile_util.cpp + ${LIBRARY_DIR}/src/nfa/mcclellancompile.cpp + ${LIBRARY_DIR}/src/nfa/mcsheng_compile.cpp + ${LIBRARY_DIR}/src/nfa/mcsheng_data.c + ${LIBRARY_DIR}/src/nfa/mcsheng.c + ${LIBRARY_DIR}/src/nfa/mpv.c + ${LIBRARY_DIR}/src/nfa/mpvcompile.cpp + ${LIBRARY_DIR}/src/nfa/nfa_api_dispatch.c + ${LIBRARY_DIR}/src/nfa/nfa_build_util.cpp + ${LIBRARY_DIR}/src/nfa/rdfa_graph.cpp + ${LIBRARY_DIR}/src/nfa/rdfa_merge.cpp + ${LIBRARY_DIR}/src/nfa/rdfa.cpp + ${LIBRARY_DIR}/src/nfa/repeat.c + ${LIBRARY_DIR}/src/nfa/repeatcompile.cpp + ${LIBRARY_DIR}/src/nfa/sheng.c + ${LIBRARY_DIR}/src/nfa/shengcompile.cpp + ${LIBRARY_DIR}/src/nfa/shufti.c + ${LIBRARY_DIR}/src/nfa/shufticompile.cpp + ${LIBRARY_DIR}/src/nfa/tamarama.c + ${LIBRARY_DIR}/src/nfa/tamaramacompile.cpp + ${LIBRARY_DIR}/src/nfa/truffle.c + ${LIBRARY_DIR}/src/nfa/trufflecompile.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_anchored_acyclic.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_anchored_dots.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_asserts.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_builder.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_calc_components.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_cyclic_redundancy.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_depth.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_dominators.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_edge_redundancy.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_equivalence.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_execute.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_expr_info.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_extparam.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_fixed_width.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_fuzzy.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_haig.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_holder.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_is_equal.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_lbr.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_limex_accel.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_limex.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_literal_analysis.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_literal_component.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_literal_decorated.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_mcclellan.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_misc_opt.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_netflow.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_prefilter.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_prune.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_puff.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_redundancy.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_region_redundancy.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_region.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_repeat.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_reports.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_restructuring.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_revacc.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_sep.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_small_literal_set.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_som_add_redundancy.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_som_util.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_som.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_split.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_squash.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_stop.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_uncalc_components.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_utf8.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_util.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_vacuous.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_violet.cpp + ${LIBRARY_DIR}/src/nfagraph/ng_width.cpp + ${LIBRARY_DIR}/src/nfagraph/ng.cpp + ${LIBRARY_DIR}/src/parser/AsciiComponentClass.cpp + ${LIBRARY_DIR}/src/parser/buildstate.cpp + ${LIBRARY_DIR}/src/parser/check_refs.cpp + ${LIBRARY_DIR}/src/parser/Component.cpp + ${LIBRARY_DIR}/src/parser/ComponentAlternation.cpp + ${LIBRARY_DIR}/src/parser/ComponentAssertion.cpp + ${LIBRARY_DIR}/src/parser/ComponentAtomicGroup.cpp + ${LIBRARY_DIR}/src/parser/ComponentBackReference.cpp + ${LIBRARY_DIR}/src/parser/ComponentBoundary.cpp + ${LIBRARY_DIR}/src/parser/ComponentByte.cpp + ${LIBRARY_DIR}/src/parser/ComponentClass.cpp + ${LIBRARY_DIR}/src/parser/ComponentCondReference.cpp + ${LIBRARY_DIR}/src/parser/ComponentEmpty.cpp + ${LIBRARY_DIR}/src/parser/ComponentEUS.cpp + ${LIBRARY_DIR}/src/parser/ComponentRepeat.cpp + ${LIBRARY_DIR}/src/parser/ComponentSequence.cpp + ${LIBRARY_DIR}/src/parser/ComponentVisitor.cpp + ${LIBRARY_DIR}/src/parser/ComponentWordBoundary.cpp + ${LIBRARY_DIR}/src/parser/ConstComponentVisitor.cpp + ${LIBRARY_DIR}/src/parser/control_verbs.cpp + ${LIBRARY_DIR}/src/parser/logical_combination.cpp + ${LIBRARY_DIR}/src/parser/parse_error.cpp + ${LIBRARY_DIR}/src/parser/parser_util.cpp + ${LIBRARY_DIR}/src/parser/Parser.cpp + ${LIBRARY_DIR}/src/parser/prefilter.cpp + ${LIBRARY_DIR}/src/parser/shortcut_literal.cpp + ${LIBRARY_DIR}/src/parser/ucp_table.cpp + ${LIBRARY_DIR}/src/parser/unsupported.cpp + ${LIBRARY_DIR}/src/parser/utf8_validate.cpp + ${LIBRARY_DIR}/src/parser/Utf8ComponentClass.cpp + ${LIBRARY_DIR}/src/rose/block.c + ${LIBRARY_DIR}/src/rose/catchup.c + ${LIBRARY_DIR}/src/rose/init.c + ${LIBRARY_DIR}/src/rose/match.c + ${LIBRARY_DIR}/src/rose/program_runtime.c + ${LIBRARY_DIR}/src/rose/rose_build_add_mask.cpp + ${LIBRARY_DIR}/src/rose/rose_build_add.cpp + ${LIBRARY_DIR}/src/rose/rose_build_anchored.cpp + ${LIBRARY_DIR}/src/rose/rose_build_bytecode.cpp + ${LIBRARY_DIR}/src/rose/rose_build_castle.cpp + ${LIBRARY_DIR}/src/rose/rose_build_compile.cpp + ${LIBRARY_DIR}/src/rose/rose_build_convert.cpp + ${LIBRARY_DIR}/src/rose/rose_build_dedupe.cpp + ${LIBRARY_DIR}/src/rose/rose_build_engine_blob.cpp + ${LIBRARY_DIR}/src/rose/rose_build_exclusive.cpp + ${LIBRARY_DIR}/src/rose/rose_build_groups.cpp + ${LIBRARY_DIR}/src/rose/rose_build_infix.cpp + ${LIBRARY_DIR}/src/rose/rose_build_instructions.cpp + ${LIBRARY_DIR}/src/rose/rose_build_lit_accel.cpp + ${LIBRARY_DIR}/src/rose/rose_build_long_lit.cpp + ${LIBRARY_DIR}/src/rose/rose_build_lookaround.cpp + ${LIBRARY_DIR}/src/rose/rose_build_matchers.cpp + ${LIBRARY_DIR}/src/rose/rose_build_merge.cpp + ${LIBRARY_DIR}/src/rose/rose_build_misc.cpp + ${LIBRARY_DIR}/src/rose/rose_build_program.cpp + ${LIBRARY_DIR}/src/rose/rose_build_role_aliasing.cpp + ${LIBRARY_DIR}/src/rose/rose_build_scatter.cpp + ${LIBRARY_DIR}/src/rose/rose_build_width.cpp + ${LIBRARY_DIR}/src/rose/rose_in_util.cpp + ${LIBRARY_DIR}/src/rose/stream.c + ${LIBRARY_DIR}/src/runtime.c + ${LIBRARY_DIR}/src/scratch.c + ${LIBRARY_DIR}/src/smallwrite/smallwrite_build.cpp + ${LIBRARY_DIR}/src/som/slot_manager.cpp + ${LIBRARY_DIR}/src/som/som_runtime.c + ${LIBRARY_DIR}/src/som/som_stream.c + ${LIBRARY_DIR}/src/stream_compress.c + ${LIBRARY_DIR}/src/util/alloc.cpp + ${LIBRARY_DIR}/src/util/charreach.cpp + ${LIBRARY_DIR}/src/util/clique.cpp + ${LIBRARY_DIR}/src/util/compile_context.cpp + ${LIBRARY_DIR}/src/util/compile_error.cpp + ${LIBRARY_DIR}/src/util/cpuid_flags.c + ${LIBRARY_DIR}/src/util/depth.cpp + ${LIBRARY_DIR}/src/util/fatbit_build.cpp + ${LIBRARY_DIR}/src/util/multibit_build.cpp + ${LIBRARY_DIR}/src/util/multibit.c + ${LIBRARY_DIR}/src/util/report_manager.cpp + ${LIBRARY_DIR}/src/util/simd_utils.c + ${LIBRARY_DIR}/src/util/state_compress.c + ${LIBRARY_DIR}/src/util/target_info.cpp + ${LIBRARY_DIR}/src/util/ue2string.cpp + ) - message (STATUS "Not using hyperscan") - return() -endif() + add_library (hyperscan ${SRCS}) -option (USE_INTERNAL_HYPERSCAN_LIBRARY "Use internal hyperscan library" ${NOT_UNBUNDLED}) - -if (NOT USE_INTERNAL_HYPERSCAN_LIBRARY) - find_library (LIBRARY_HYPERSCAN hs) - find_path (INCLUDE_HYPERSCAN NAMES hs.h HINTS /usr/include/hs) # Ubuntu puts headers in this folder - - if (LIBRARY_HYPERSCAN AND INCLUDE_HYPERSCAN) - set (EXTERNAL_HYPERSCAN_LIBRARY_FOUND 1) + target_compile_definitions (hyperscan PUBLIC USE_HYPERSCAN=1) + target_compile_options (hyperscan + PRIVATE -g0 # Library has too much debug information + -march=corei7 -O2 -fno-strict-aliasing -fno-omit-frame-pointer -fvisibility=hidden # The options from original build system + -fno-sanitize=undefined # Assume the library takes care of itself + ) + target_include_directories (hyperscan + PRIVATE + common + ${LIBRARY_DIR}/include + ) + target_include_directories (hyperscan SYSTEM PUBLIC ${LIBRARY_DIR}/src) + if (ARCH_AMD64) + target_include_directories (hyperscan PRIVATE x86_64) + endif () + target_link_libraries (hyperscan PRIVATE boost::headers_only) + else () + find_library (LIBRARY_HYPERSCAN hs) + find_path (INCLUDE_HYPERSCAN NAMES hs.h HINTS /usr/include/hs) # Ubuntu puts headers in this folder add_library (hyperscan UNKNOWN IMPORTED GLOBAL) set_target_properties (hyperscan PROPERTIES IMPORTED_LOCATION ${LIBRARY_HYPERSCAN}) set_target_properties (hyperscan PROPERTIES INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_HYPERSCAN}) set_property(TARGET hyperscan APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS USE_HYPERSCAN=1) - else () - set (EXTERNAL_HYPERSCAN_LIBRARY_FOUND 0) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system hyperscan library") endif () + message (STATUS "Using hyperscan") +else () + add_library (hyperscan INTERFACE) + target_compile_definitions (hyperscan INTERFACE USE_HYPERSCAN=0) + + message (STATUS "Not using hyperscan") endif () - -if (NOT EXTERNAL_HYPERSCAN_LIBRARY_FOUND) - set (USE_INTERNAL_HYPERSCAN_LIBRARY 1) - - set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/hyperscan) - - set (SRCS - ${LIBRARY_DIR}/src/alloc.c - ${LIBRARY_DIR}/src/compiler/asserts.cpp - ${LIBRARY_DIR}/src/compiler/compiler.cpp - ${LIBRARY_DIR}/src/compiler/error.cpp - ${LIBRARY_DIR}/src/crc32.c - ${LIBRARY_DIR}/src/database.c - ${LIBRARY_DIR}/src/fdr/engine_description.cpp - ${LIBRARY_DIR}/src/fdr/fdr_compile_util.cpp - ${LIBRARY_DIR}/src/fdr/fdr_compile.cpp - ${LIBRARY_DIR}/src/fdr/fdr_confirm_compile.cpp - ${LIBRARY_DIR}/src/fdr/fdr_engine_description.cpp - ${LIBRARY_DIR}/src/fdr/fdr.c - ${LIBRARY_DIR}/src/fdr/flood_compile.cpp - ${LIBRARY_DIR}/src/fdr/teddy_compile.cpp - ${LIBRARY_DIR}/src/fdr/teddy_engine_description.cpp - ${LIBRARY_DIR}/src/fdr/teddy.c - ${LIBRARY_DIR}/src/grey.cpp - ${LIBRARY_DIR}/src/hs_valid_platform.c - ${LIBRARY_DIR}/src/hs_version.c - ${LIBRARY_DIR}/src/hs.cpp - ${LIBRARY_DIR}/src/hwlm/hwlm_build.cpp - ${LIBRARY_DIR}/src/hwlm/hwlm_literal.cpp - ${LIBRARY_DIR}/src/hwlm/hwlm.c - ${LIBRARY_DIR}/src/hwlm/noodle_build.cpp - ${LIBRARY_DIR}/src/hwlm/noodle_engine.c - ${LIBRARY_DIR}/src/nfa/accel_dfa_build_strat.cpp - ${LIBRARY_DIR}/src/nfa/accel.c - ${LIBRARY_DIR}/src/nfa/accelcompile.cpp - ${LIBRARY_DIR}/src/nfa/castle.c - ${LIBRARY_DIR}/src/nfa/castlecompile.cpp - ${LIBRARY_DIR}/src/nfa/dfa_build_strat.cpp - ${LIBRARY_DIR}/src/nfa/dfa_min.cpp - ${LIBRARY_DIR}/src/nfa/gough.c - ${LIBRARY_DIR}/src/nfa/goughcompile_accel.cpp - ${LIBRARY_DIR}/src/nfa/goughcompile_reg.cpp - ${LIBRARY_DIR}/src/nfa/goughcompile.cpp - ${LIBRARY_DIR}/src/nfa/lbr.c - ${LIBRARY_DIR}/src/nfa/limex_64.c - ${LIBRARY_DIR}/src/nfa/limex_accel.c - ${LIBRARY_DIR}/src/nfa/limex_compile.cpp - ${LIBRARY_DIR}/src/nfa/limex_native.c - ${LIBRARY_DIR}/src/nfa/limex_simd128.c - ${LIBRARY_DIR}/src/nfa/limex_simd256.c - ${LIBRARY_DIR}/src/nfa/limex_simd384.c - ${LIBRARY_DIR}/src/nfa/limex_simd512.c - ${LIBRARY_DIR}/src/nfa/mcclellan.c - ${LIBRARY_DIR}/src/nfa/mcclellancompile_util.cpp - ${LIBRARY_DIR}/src/nfa/mcclellancompile.cpp - ${LIBRARY_DIR}/src/nfa/mcsheng_compile.cpp - ${LIBRARY_DIR}/src/nfa/mcsheng_data.c - ${LIBRARY_DIR}/src/nfa/mcsheng.c - ${LIBRARY_DIR}/src/nfa/mpv.c - ${LIBRARY_DIR}/src/nfa/mpvcompile.cpp - ${LIBRARY_DIR}/src/nfa/nfa_api_dispatch.c - ${LIBRARY_DIR}/src/nfa/nfa_build_util.cpp - ${LIBRARY_DIR}/src/nfa/rdfa_graph.cpp - ${LIBRARY_DIR}/src/nfa/rdfa_merge.cpp - ${LIBRARY_DIR}/src/nfa/rdfa.cpp - ${LIBRARY_DIR}/src/nfa/repeat.c - ${LIBRARY_DIR}/src/nfa/repeatcompile.cpp - ${LIBRARY_DIR}/src/nfa/sheng.c - ${LIBRARY_DIR}/src/nfa/shengcompile.cpp - ${LIBRARY_DIR}/src/nfa/shufti.c - ${LIBRARY_DIR}/src/nfa/shufticompile.cpp - ${LIBRARY_DIR}/src/nfa/tamarama.c - ${LIBRARY_DIR}/src/nfa/tamaramacompile.cpp - ${LIBRARY_DIR}/src/nfa/truffle.c - ${LIBRARY_DIR}/src/nfa/trufflecompile.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_anchored_acyclic.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_anchored_dots.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_asserts.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_builder.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_calc_components.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_cyclic_redundancy.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_depth.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_dominators.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_edge_redundancy.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_equivalence.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_execute.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_expr_info.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_extparam.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_fixed_width.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_fuzzy.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_haig.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_holder.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_is_equal.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_lbr.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_limex_accel.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_limex.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_literal_analysis.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_literal_component.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_literal_decorated.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_mcclellan.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_misc_opt.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_netflow.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_prefilter.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_prune.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_puff.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_redundancy.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_region_redundancy.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_region.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_repeat.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_reports.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_restructuring.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_revacc.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_sep.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_small_literal_set.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_som_add_redundancy.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_som_util.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_som.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_split.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_squash.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_stop.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_uncalc_components.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_utf8.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_util.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_vacuous.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_violet.cpp - ${LIBRARY_DIR}/src/nfagraph/ng_width.cpp - ${LIBRARY_DIR}/src/nfagraph/ng.cpp - ${LIBRARY_DIR}/src/parser/AsciiComponentClass.cpp - ${LIBRARY_DIR}/src/parser/buildstate.cpp - ${LIBRARY_DIR}/src/parser/check_refs.cpp - ${LIBRARY_DIR}/src/parser/Component.cpp - ${LIBRARY_DIR}/src/parser/ComponentAlternation.cpp - ${LIBRARY_DIR}/src/parser/ComponentAssertion.cpp - ${LIBRARY_DIR}/src/parser/ComponentAtomicGroup.cpp - ${LIBRARY_DIR}/src/parser/ComponentBackReference.cpp - ${LIBRARY_DIR}/src/parser/ComponentBoundary.cpp - ${LIBRARY_DIR}/src/parser/ComponentByte.cpp - ${LIBRARY_DIR}/src/parser/ComponentClass.cpp - ${LIBRARY_DIR}/src/parser/ComponentCondReference.cpp - ${LIBRARY_DIR}/src/parser/ComponentEmpty.cpp - ${LIBRARY_DIR}/src/parser/ComponentEUS.cpp - ${LIBRARY_DIR}/src/parser/ComponentRepeat.cpp - ${LIBRARY_DIR}/src/parser/ComponentSequence.cpp - ${LIBRARY_DIR}/src/parser/ComponentVisitor.cpp - ${LIBRARY_DIR}/src/parser/ComponentWordBoundary.cpp - ${LIBRARY_DIR}/src/parser/ConstComponentVisitor.cpp - ${LIBRARY_DIR}/src/parser/control_verbs.cpp - ${LIBRARY_DIR}/src/parser/logical_combination.cpp - ${LIBRARY_DIR}/src/parser/parse_error.cpp - ${LIBRARY_DIR}/src/parser/parser_util.cpp - ${LIBRARY_DIR}/src/parser/Parser.cpp - ${LIBRARY_DIR}/src/parser/prefilter.cpp - ${LIBRARY_DIR}/src/parser/shortcut_literal.cpp - ${LIBRARY_DIR}/src/parser/ucp_table.cpp - ${LIBRARY_DIR}/src/parser/unsupported.cpp - ${LIBRARY_DIR}/src/parser/utf8_validate.cpp - ${LIBRARY_DIR}/src/parser/Utf8ComponentClass.cpp - ${LIBRARY_DIR}/src/rose/block.c - ${LIBRARY_DIR}/src/rose/catchup.c - ${LIBRARY_DIR}/src/rose/init.c - ${LIBRARY_DIR}/src/rose/match.c - ${LIBRARY_DIR}/src/rose/program_runtime.c - ${LIBRARY_DIR}/src/rose/rose_build_add_mask.cpp - ${LIBRARY_DIR}/src/rose/rose_build_add.cpp - ${LIBRARY_DIR}/src/rose/rose_build_anchored.cpp - ${LIBRARY_DIR}/src/rose/rose_build_bytecode.cpp - ${LIBRARY_DIR}/src/rose/rose_build_castle.cpp - ${LIBRARY_DIR}/src/rose/rose_build_compile.cpp - ${LIBRARY_DIR}/src/rose/rose_build_convert.cpp - ${LIBRARY_DIR}/src/rose/rose_build_dedupe.cpp - ${LIBRARY_DIR}/src/rose/rose_build_engine_blob.cpp - ${LIBRARY_DIR}/src/rose/rose_build_exclusive.cpp - ${LIBRARY_DIR}/src/rose/rose_build_groups.cpp - ${LIBRARY_DIR}/src/rose/rose_build_infix.cpp - ${LIBRARY_DIR}/src/rose/rose_build_instructions.cpp - ${LIBRARY_DIR}/src/rose/rose_build_lit_accel.cpp - ${LIBRARY_DIR}/src/rose/rose_build_long_lit.cpp - ${LIBRARY_DIR}/src/rose/rose_build_lookaround.cpp - ${LIBRARY_DIR}/src/rose/rose_build_matchers.cpp - ${LIBRARY_DIR}/src/rose/rose_build_merge.cpp - ${LIBRARY_DIR}/src/rose/rose_build_misc.cpp - ${LIBRARY_DIR}/src/rose/rose_build_program.cpp - ${LIBRARY_DIR}/src/rose/rose_build_role_aliasing.cpp - ${LIBRARY_DIR}/src/rose/rose_build_scatter.cpp - ${LIBRARY_DIR}/src/rose/rose_build_width.cpp - ${LIBRARY_DIR}/src/rose/rose_in_util.cpp - ${LIBRARY_DIR}/src/rose/stream.c - ${LIBRARY_DIR}/src/runtime.c - ${LIBRARY_DIR}/src/scratch.c - ${LIBRARY_DIR}/src/smallwrite/smallwrite_build.cpp - ${LIBRARY_DIR}/src/som/slot_manager.cpp - ${LIBRARY_DIR}/src/som/som_runtime.c - ${LIBRARY_DIR}/src/som/som_stream.c - ${LIBRARY_DIR}/src/stream_compress.c - ${LIBRARY_DIR}/src/util/alloc.cpp - ${LIBRARY_DIR}/src/util/charreach.cpp - ${LIBRARY_DIR}/src/util/clique.cpp - ${LIBRARY_DIR}/src/util/compile_context.cpp - ${LIBRARY_DIR}/src/util/compile_error.cpp - ${LIBRARY_DIR}/src/util/cpuid_flags.c - ${LIBRARY_DIR}/src/util/depth.cpp - ${LIBRARY_DIR}/src/util/fatbit_build.cpp - ${LIBRARY_DIR}/src/util/multibit_build.cpp - ${LIBRARY_DIR}/src/util/multibit.c - ${LIBRARY_DIR}/src/util/report_manager.cpp - ${LIBRARY_DIR}/src/util/simd_utils.c - ${LIBRARY_DIR}/src/util/state_compress.c - ${LIBRARY_DIR}/src/util/target_info.cpp - ${LIBRARY_DIR}/src/util/ue2string.cpp - ) - - add_library (hyperscan ${SRCS}) - - target_compile_definitions (hyperscan PUBLIC USE_HYPERSCAN=1) - target_compile_options (hyperscan - PRIVATE -g0 # Library has too much debug information - -march=corei7 -O2 -fno-strict-aliasing -fno-omit-frame-pointer -fvisibility=hidden # The options from original build system - -fno-sanitize=undefined # Assume the library takes care of itself - ) - target_include_directories (hyperscan - PRIVATE - common - ${LIBRARY_DIR}/include - ) - target_include_directories (hyperscan SYSTEM PUBLIC ${LIBRARY_DIR}/src) - if (ARCH_AMD64) - target_include_directories (hyperscan PRIVATE x86_64) - endif () - target_link_libraries (hyperscan PRIVATE boost::headers_only) - - set (USE_INTERNAL_HYPERSCAN_LIBRARY 1) -endif () - -message (STATUS "Using hyperscan") diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index e8a690ff149..13f7ea3326b 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -2,40 +2,140 @@ option (ENABLE_JEMALLOC "Enable jemalloc allocator" ${ENABLE_LIBRARIES}) if (SANITIZE OR NOT (ARCH_AMD64 OR ARCH_ARM) OR NOT (OS_LINUX OR OS_FREEBSD OR OS_DARWIN)) set (ENABLE_JEMALLOC OFF) - message (${RECONFIGURE_MESSAGE_LEVEL} - "jemalloc is disabled implicitly: it doesn't work with sanitizers and can only be used with x86_64 or aarch64 on linux or freebsd.") + message (STATUS "jemalloc is disabled implicitly: it doesn't work with sanitizers and can only be used with x86_64 or aarch64 on linux or freebsd.") endif () -if (NOT ENABLE_JEMALLOC) - if(USE_INTERNAL_JEMALLOC_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal jemalloc with ENABLE_JEMALLOC=OFF") +if (ENABLE_JEMALLOC) + if (NOT OS_LINUX) + message (WARNING "jemalloc support on non-linux is EXPERIMENTAL") endif() - add_library(jemalloc INTERFACE) - target_compile_definitions(jemalloc INTERFACE USE_JEMALLOC=0) + option (USE_INTERNAL_JEMALLOC "Use internal jemalloc library" ${NOT_UNBUNDLED}) - message (STATUS "Not using jemalloc") - return() -endif () + if (USE_INTERNAL_JEMALLOC) + if (OS_LINUX) + # ThreadPool select job randomly, and there can be some threads that had been + # performed some memory heavy task before and will be inactive for some time, + # but until it will became active again, the memory will not be freed since by + # default each thread has it's own arena, but there should be not more then + # 4*CPU arenas (see opt.nareans description). + # + # By enabling percpu_arena number of arenas limited to number of CPUs and hence + # this problem should go away. + # + # muzzy_decay_ms -- use MADV_FREE when available on newer Linuxes, to + # avoid spurious latencies and additional work associated with + # MADV_DONTNEED. See + # https://github.com/ClickHouse/ClickHouse/issues/11121 for motivation. + set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0,muzzy_decay_ms:10000") + else() + set (JEMALLOC_CONFIG_MALLOC_CONF "oversize_threshold:0,muzzy_decay_ms:10000") + endif() + # CACHE variable is empty, to allow changing defaults without necessity + # to purge cache + set (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE "" CACHE STRING "Change default configuration string of JEMalloc" ) + if (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE) + set (JEMALLOC_CONFIG_MALLOC_CONF "${JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE}") + endif() + message (STATUS "jemalloc malloc_conf: ${JEMALLOC_CONFIG_MALLOC_CONF}") -if (NOT OS_LINUX) - message (WARNING "jemalloc support on non-linux is EXPERIMENTAL") -endif() + set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/jemalloc") -option (USE_INTERNAL_JEMALLOC_LIBRARY "Use internal jemalloc library" ${NOT_UNBUNDLED}) + set (SRCS + ${LIBRARY_DIR}/src/arena.c + ${LIBRARY_DIR}/src/background_thread.c + ${LIBRARY_DIR}/src/base.c + ${LIBRARY_DIR}/src/bin.c + ${LIBRARY_DIR}/src/bitmap.c + ${LIBRARY_DIR}/src/ckh.c + ${LIBRARY_DIR}/src/ctl.c + ${LIBRARY_DIR}/src/div.c + ${LIBRARY_DIR}/src/extent.c + ${LIBRARY_DIR}/src/extent_dss.c + ${LIBRARY_DIR}/src/extent_mmap.c + ${LIBRARY_DIR}/src/hash.c + ${LIBRARY_DIR}/src/hook.c + ${LIBRARY_DIR}/src/jemalloc.c + ${LIBRARY_DIR}/src/large.c + ${LIBRARY_DIR}/src/log.c + ${LIBRARY_DIR}/src/malloc_io.c + ${LIBRARY_DIR}/src/mutex.c + ${LIBRARY_DIR}/src/mutex_pool.c + ${LIBRARY_DIR}/src/nstime.c + ${LIBRARY_DIR}/src/pages.c + ${LIBRARY_DIR}/src/prng.c + ${LIBRARY_DIR}/src/prof.c + ${LIBRARY_DIR}/src/rtree.c + ${LIBRARY_DIR}/src/sc.c + ${LIBRARY_DIR}/src/stats.c + ${LIBRARY_DIR}/src/sz.c + ${LIBRARY_DIR}/src/tcache.c + ${LIBRARY_DIR}/src/test_hooks.c + ${LIBRARY_DIR}/src/ticker.c + ${LIBRARY_DIR}/src/tsd.c + ${LIBRARY_DIR}/src/witness.c + ${LIBRARY_DIR}/src/safety_check.c + ) + if (OS_DARWIN) + list(APPEND SRCS ${LIBRARY_DIR}/src/zone.c) + endif () -if (NOT USE_INTERNAL_JEMALLOC_LIBRARY) - find_library(LIBRARY_JEMALLOC jemalloc) - find_path(INCLUDE_JEMALLOC jemalloc/jemalloc.h) + add_library(jemalloc ${SRCS}) + target_include_directories(jemalloc PRIVATE ${LIBRARY_DIR}/include) + target_include_directories(jemalloc SYSTEM PUBLIC include) - if (LIBRARY_JEMALLOC AND INCLUDE_JEMALLOC) - set(EXTERNAL_JEMALLOC_LIBRARY_FOUND 1) + set (JEMALLOC_INCLUDE_PREFIX) + # OS_ + if (OS_LINUX) + set (JEMALLOC_INCLUDE_PREFIX "include_linux") + elseif (OS_FREEBSD) + set (JEMALLOC_INCLUDE_PREFIX "include_freebsd") + elseif (OS_DARWIN) + set (JEMALLOC_INCLUDE_PREFIX "include_darwin") + else () + message (FATAL_ERROR "This OS is not supported") + endif () + # ARCH_ + if (ARCH_AMD64) + set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_x86_64") + elseif (ARCH_ARM) + set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_aarch64") + else () + message (FATAL_ERROR "This arch is not supported") + endif () + + configure_file(${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal/jemalloc_internal_defs.h.in + ${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal/jemalloc_internal_defs.h) + target_include_directories(jemalloc SYSTEM PRIVATE + ${CMAKE_CURRENT_BINARY_DIR}/${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal) + + target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE) + + if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") + target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_DEBUG=1 -DJEMALLOC_PROF=1) + + if (USE_UNWIND) + target_compile_definitions (jemalloc PRIVATE -DJEMALLOC_PROF_LIBUNWIND=1) + target_link_libraries (jemalloc PRIVATE unwind) + endif () + endif () + + target_compile_options(jemalloc PRIVATE -Wno-redundant-decls) + # for RTLD_NEXT + target_compile_options(jemalloc PRIVATE -D_GNU_SOURCE) + else () + find_library(LIBRARY_JEMALLOC jemalloc) + find_path(INCLUDE_JEMALLOC jemalloc/jemalloc.h) set(THREADS_PREFER_PTHREAD_FLAG ON) - find_package(Threads) + find_package(Threads REQUIRED) - set (CMAKE_REQUIRED_LIBRARIES ${LIBRARY_JEMALLOC} Threads::Threads "dl") - set (CMAKE_REQUIRED_INCLUDES ${INCLUDE_JEMALLOC}) + add_library (jemalloc STATIC IMPORTED) + set_property (TARGET jemalloc PROPERTY IMPORTED_LOCATION ${LIBRARY_JEMALLOC}) + set_property (TARGET jemalloc PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_JEMALLOC}) + set_property (TARGET jemalloc PROPERTY INTERFACE_LINK_LIBRARIES Threads::Threads dl) + + set (CMAKE_REQUIRED_LIBRARIES jemalloc) check_cxx_source_compiles ( " #include <jemalloc/jemalloc.h> @@ -44,145 +144,24 @@ if (NOT USE_INTERNAL_JEMALLOC_LIBRARY) free(mallocx(1, 0)); } " - EXTERNAL_JEMALLOC_LIBRARY_WORKS + EXTERNAL_JEMALLOC_WORKS ) - if (EXTERNAL_JEMALLOC_LIBRARY_WORKS) - add_library (jemalloc STATIC IMPORTED) - set_property (TARGET jemalloc PROPERTY IMPORTED_LOCATION ${LIBRARY_JEMALLOC}) - set_property (TARGET jemalloc PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_JEMALLOC}) - set_property (TARGET jemalloc PROPERTY INTERFACE_LINK_LIBRARIES Threads::Threads dl) - else() - message (${RECONFIGURE_MESSAGE_LEVEL} "External jemalloc is unusable: ${LIBRARY_JEMALLOC} ${INCLUDE_JEMALLOC}") - endif () - - else() - set(EXTERNAL_JEMALLOC_LIBRARY_FOUND 0) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system jemalloc") - endif() -endif () - -if (NOT EXTERNAL_JEMALLOC_LIBRARY_FOUND OR NOT EXTERNAL_JEMALLOC_LIBRARY_WORKS) - set(USE_INTERNAL_JEMALLOC_LIBRARY 1) - - if (OS_LINUX) - # ThreadPool select job randomly, and there can be some threads that had been - # performed some memory heavy task before and will be inactive for some time, - # but until it will became active again, the memory will not be freed since by - # default each thread has it's own arena, but there should be not more then - # 4*CPU arenas (see opt.nareans description). - # - # By enabling percpu_arena number of arenas limited to number of CPUs and hence - # this problem should go away. - # - # muzzy_decay_ms -- use MADV_FREE when available on newer Linuxes, to - # avoid spurious latencies and additional work associated with - # MADV_DONTNEED. See - # https://github.com/ClickHouse/ClickHouse/issues/11121 for motivation. - set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0,muzzy_decay_ms:10000") - else() - set (JEMALLOC_CONFIG_MALLOC_CONF "oversize_threshold:0,muzzy_decay_ms:10000") - endif() - # CACHE variable is empty, to allow changing defaults without necessity - # to purge cache - set (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE "" CACHE STRING "Change default configuration string of JEMalloc" ) - if (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE) - set (JEMALLOC_CONFIG_MALLOC_CONF "${JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE}") - endif() - message (STATUS "jemalloc malloc_conf: ${JEMALLOC_CONFIG_MALLOC_CONF}") - - set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/jemalloc") - - set (SRCS - ${LIBRARY_DIR}/src/arena.c - ${LIBRARY_DIR}/src/background_thread.c - ${LIBRARY_DIR}/src/base.c - ${LIBRARY_DIR}/src/bin.c - ${LIBRARY_DIR}/src/bitmap.c - ${LIBRARY_DIR}/src/ckh.c - ${LIBRARY_DIR}/src/ctl.c - ${LIBRARY_DIR}/src/div.c - ${LIBRARY_DIR}/src/extent.c - ${LIBRARY_DIR}/src/extent_dss.c - ${LIBRARY_DIR}/src/extent_mmap.c - ${LIBRARY_DIR}/src/hash.c - ${LIBRARY_DIR}/src/hook.c - ${LIBRARY_DIR}/src/jemalloc.c - ${LIBRARY_DIR}/src/large.c - ${LIBRARY_DIR}/src/log.c - ${LIBRARY_DIR}/src/malloc_io.c - ${LIBRARY_DIR}/src/mutex.c - ${LIBRARY_DIR}/src/mutex_pool.c - ${LIBRARY_DIR}/src/nstime.c - ${LIBRARY_DIR}/src/pages.c - ${LIBRARY_DIR}/src/prng.c - ${LIBRARY_DIR}/src/prof.c - ${LIBRARY_DIR}/src/rtree.c - ${LIBRARY_DIR}/src/sc.c - ${LIBRARY_DIR}/src/stats.c - ${LIBRARY_DIR}/src/sz.c - ${LIBRARY_DIR}/src/tcache.c - ${LIBRARY_DIR}/src/test_hooks.c - ${LIBRARY_DIR}/src/ticker.c - ${LIBRARY_DIR}/src/tsd.c - ${LIBRARY_DIR}/src/witness.c - ${LIBRARY_DIR}/src/safety_check.c - ) - if (OS_DARWIN) - list(APPEND SRCS ${LIBRARY_DIR}/src/zone.c) - endif () - - add_library(jemalloc ${SRCS}) - target_include_directories(jemalloc PRIVATE ${LIBRARY_DIR}/include) - target_include_directories(jemalloc SYSTEM PUBLIC include) - - set (JEMALLOC_INCLUDE_PREFIX) - # OS_ - if (OS_LINUX) - set (JEMALLOC_INCLUDE_PREFIX "include_linux") - elseif (OS_FREEBSD) - set (JEMALLOC_INCLUDE_PREFIX "include_freebsd") - elseif (OS_DARWIN) - set (JEMALLOC_INCLUDE_PREFIX "include_darwin") - else () - message (FATAL_ERROR "internal jemalloc: This OS is not supported") - endif () - # ARCH_ - if (ARCH_AMD64) - set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_x86_64") - elseif (ARCH_ARM) - set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_aarch64") - else () - message (FATAL_ERROR "internal jemalloc: This arch is not supported") - endif () - - configure_file(${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal/jemalloc_internal_defs.h.in - ${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal/jemalloc_internal_defs.h) - target_include_directories(jemalloc SYSTEM PRIVATE - ${CMAKE_CURRENT_BINARY_DIR}/${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal) - - target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE) - - if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") - target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_DEBUG=1 -DJEMALLOC_PROF=1) - - if (USE_UNWIND) - target_compile_definitions (jemalloc PRIVATE -DJEMALLOC_PROF_LIBUNWIND=1) - target_link_libraries (jemalloc PRIVATE unwind) + if (NOT EXTERNAL_JEMALLOC_WORKS) + message (FATAL_ERROR "jemalloc is unusable: ${LIBRARY_JEMALLOC} ${INCLUDE_JEMALLOC}") endif () endif () - target_compile_options(jemalloc PRIVATE -Wno-redundant-decls) - # for RTLD_NEXT - target_compile_options(jemalloc PRIVATE -D_GNU_SOURCE) + set_property(TARGET jemalloc APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS USE_JEMALLOC=1) + if (MAKE_STATIC_LIBRARIES) + # To detect whether we need to register jemalloc for osx as default zone. + set_property(TARGET jemalloc APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS BUNDLED_STATIC_JEMALLOC=1) + endif() - set (USE_INTERNAL_JEMALLOC_LIBRARY 1) + message (STATUS "Using jemalloc") +else () + add_library(jemalloc INTERFACE) + target_compile_definitions(jemalloc INTERFACE USE_JEMALLOC=0) + + message (STATUS "Not using jemalloc") endif () - -set_property(TARGET jemalloc APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS USE_JEMALLOC=1) -if (MAKE_STATIC_LIBRARIES) - # To detect whether we need to register jemalloc for osx as default zone. - set_property(TARGET jemalloc APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS BUNDLED_STATIC_JEMALLOC=1) -endif() - -message (STATUS "Using jemalloc") diff --git a/contrib/libcpuid-cmake/CMakeLists.txt b/contrib/libcpuid-cmake/CMakeLists.txt index 564e67994bf..cb28cbd21da 100644 --- a/contrib/libcpuid-cmake/CMakeLists.txt +++ b/contrib/libcpuid-cmake/CMakeLists.txt @@ -1,38 +1,35 @@ option (ENABLE_CPUID "Enable libcpuid library (only internal)" ${ENABLE_LIBRARIES}) -if (ARCH_ARM AND ENABLE_CPUID) - message (${RECONFIGURE_MESSAGE_LEVEL} "cpuid is not supported on ARM") +if (ARCH_ARM) set (ENABLE_CPUID 0) endif () -if (NOT ENABLE_CPUID) +if (ENABLE_CPUID) + set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/libcpuid) + + set (SRCS + ${LIBRARY_DIR}/libcpuid/asm-bits.c + ${LIBRARY_DIR}/libcpuid/cpuid_main.c + ${LIBRARY_DIR}/libcpuid/libcpuid_util.c + ${LIBRARY_DIR}/libcpuid/msrdriver.c + ${LIBRARY_DIR}/libcpuid/rdmsr.c + ${LIBRARY_DIR}/libcpuid/rdtsc.c + ${LIBRARY_DIR}/libcpuid/recog_amd.c + ${LIBRARY_DIR}/libcpuid/recog_intel.c + ) + + add_library (cpuid ${SRCS}) + + target_include_directories (cpuid SYSTEM PUBLIC ${LIBRARY_DIR}) + target_compile_definitions (cpuid PUBLIC USE_CPUID=1) + target_compile_definitions (cpuid PRIVATE VERSION="v0.4.1") + if (COMPILER_CLANG) + target_compile_options (cpuid PRIVATE -Wno-reserved-id-macro) + endif () + + message (STATUS "Using cpuid") +else () add_library (cpuid INTERFACE) target_compile_definitions (cpuid INTERFACE USE_CPUID=0) - - return() -endif() - -set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/libcpuid") - -set (SRCS - ${LIBRARY_DIR}/libcpuid/asm-bits.c - ${LIBRARY_DIR}/libcpuid/cpuid_main.c - ${LIBRARY_DIR}/libcpuid/libcpuid_util.c - ${LIBRARY_DIR}/libcpuid/msrdriver.c - ${LIBRARY_DIR}/libcpuid/rdmsr.c - ${LIBRARY_DIR}/libcpuid/rdtsc.c - ${LIBRARY_DIR}/libcpuid/recog_amd.c - ${LIBRARY_DIR}/libcpuid/recog_intel.c -) - -add_library (cpuid ${SRCS}) - -target_include_directories (cpuid SYSTEM PUBLIC ${LIBRARY_DIR}) -target_compile_definitions (cpuid PUBLIC USE_CPUID=1) -target_compile_definitions (cpuid PRIVATE VERSION="v0.4.1") -if (COMPILER_CLANG) - target_compile_options (cpuid PRIVATE -Wno-reserved-id-macro) endif () - -message (STATUS "Using cpuid") diff --git a/contrib/libhdfs3-cmake/CMakeLists.txt b/contrib/libhdfs3-cmake/CMakeLists.txt index 5b704ab1410..4c71770f5b6 100644 --- a/contrib/libhdfs3-cmake/CMakeLists.txt +++ b/contrib/libhdfs3-cmake/CMakeLists.txt @@ -1,12 +1,4 @@ -option(PROTOBUF_OLD_ABI_COMPAT "Set to ON for compatiability with external protobuf which was compiled old C++ ABI" ON) - -if (PROTOBUF_OLD_ABI_COMPAT) - if (NOT ENABLE_PROTOBUF OR USE_INTERNAL_PROTOBUF_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "PROTOBUF_OLD_ABI_COMPAT option is ignored") - endif() -endif() - -if (NOT USE_INTERNAL_PROTOBUF_LIBRARY AND PROTOBUF_OLD_ABI_COMPAT) +if (NOT USE_INTERNAL_PROTOBUF_LIBRARY) # compatiable with protobuf which was compiled old C++ ABI set(CMAKE_CXX_FLAGS "-D_GLIBCXX_USE_CXX11_ABI=0") set(CMAKE_C_FLAGS "") @@ -15,7 +7,7 @@ if (NOT USE_INTERNAL_PROTOBUF_LIBRARY AND PROTOBUF_OLD_ABI_COMPAT) endif () endif() -set(WITH_KERBEROS false) +SET(WITH_KERBEROS false) # project and source dir set(HDFS3_ROOT_DIR ${ClickHouse_SOURCE_DIR}/contrib/libhdfs3) set(HDFS3_SOURCE_DIR ${HDFS3_ROOT_DIR}/src) diff --git a/contrib/lz4-cmake/CMakeLists.txt b/contrib/lz4-cmake/CMakeLists.txt index 374837fbe58..b8121976213 100644 --- a/contrib/lz4-cmake/CMakeLists.txt +++ b/contrib/lz4-cmake/CMakeLists.txt @@ -1,30 +1,13 @@ option (USE_INTERNAL_LZ4_LIBRARY "Use internal lz4 library" ${NOT_UNBUNDLED}) -if (NOT USE_INTERNAL_LZ4_LIBRARY) - find_library (LIBRARY_LZ4 lz4) - find_path (INCLUDE_LZ4 lz4.h) - - if (LIBRARY_LZ4 AND INCLUDE_LZ4) - set(EXTERNAL_LZ4_LIBRARY_FOUND 1) - add_library (lz4 UNKNOWN IMPORTED) - set_property (TARGET lz4 PROPERTY IMPORTED_LOCATION ${LIBRARY_LZ4}) - set_property (TARGET lz4 PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_LZ4}) - set_property (TARGET lz4 APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS USE_XXHASH=0) - else() - set(EXTERNAL_LZ4_LIBRARY_FOUND 0) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system lz4") - endif() -endif() - -if (NOT EXTERNAL_LZ4_LIBRARY_FOUND) - set (USE_INTERNAL_LZ4_LIBRARY 1) - set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/lz4") +if (USE_INTERNAL_LZ4_LIBRARY) + set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/lz4) set (SRCS - "${LIBRARY_DIR}/lib/lz4.c" - "${LIBRARY_DIR}/lib/lz4hc.c" - "${LIBRARY_DIR}/lib/lz4frame.c" - "${LIBRARY_DIR}/lib/xxhash.c" + ${LIBRARY_DIR}/lib/lz4.c + ${LIBRARY_DIR}/lib/lz4hc.c + ${LIBRARY_DIR}/lib/lz4frame.c + ${LIBRARY_DIR}/lib/xxhash.c ) add_library (lz4 ${SRCS}) @@ -34,4 +17,12 @@ if (NOT EXTERNAL_LZ4_LIBRARY_FOUND) target_compile_options (lz4 PRIVATE -fno-sanitize=undefined) endif () target_include_directories(lz4 PUBLIC ${LIBRARY_DIR}/lib) +else () + find_library (LIBRARY_LZ4 lz4) + find_path (INCLUDE_LZ4 lz4.h) + + add_library (lz4 UNKNOWN IMPORTED) + set_property (TARGET lz4 PROPERTY IMPORTED_LOCATION ${LIBRARY_LZ4}) + set_property (TARGET lz4 PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_LZ4}) + set_property (TARGET lz4 APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS USE_XXHASH=0) endif () diff --git a/contrib/poco-cmake/CMakeLists.txt b/contrib/poco-cmake/CMakeLists.txt index a96b07085ba..59b6c84a1d1 100644 --- a/contrib/poco-cmake/CMakeLists.txt +++ b/contrib/poco-cmake/CMakeLists.txt @@ -4,9 +4,6 @@ if (USE_INTERNAL_POCO_LIBRARY) set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/poco) else () find_path (ROOT_DIR NAMES Foundation/include/Poco/Poco.h include/Poco/Poco.h) - if (NOT ROOT_DIR) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system poco") - endif() endif () add_subdirectory (Crypto) diff --git a/contrib/replxx-cmake/CMakeLists.txt b/contrib/replxx-cmake/CMakeLists.txt index ed6bdaa25bc..48d7e8bb36b 100644 --- a/contrib/replxx-cmake/CMakeLists.txt +++ b/contrib/replxx-cmake/CMakeLists.txt @@ -1,26 +1,35 @@ option (ENABLE_REPLXX "Enable replxx support" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_REPLXX) - if (USE_INTERNAL_REPLXX_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal replxx with ENABLE_REPLXX=OFF") - endif() +if (ENABLE_REPLXX) + option (USE_INTERNAL_REPLXX "Use internal replxx library" ${NOT_UNBUNDLED}) - add_library(replxx INTERFACE) - target_compile_definitions(replxx INTERFACE USE_REPLXX=0) + if (USE_INTERNAL_REPLXX) + set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/replxx") - message (STATUS "Not using replxx (Beware! Runtime fallback to readline is possible!)") - return() -endif() + set(SRCS + ${LIBRARY_DIR}/src/conversion.cxx + ${LIBRARY_DIR}/src/ConvertUTF.cpp + ${LIBRARY_DIR}/src/escape.cxx + ${LIBRARY_DIR}/src/history.cxx + ${LIBRARY_DIR}/src/io.cxx + ${LIBRARY_DIR}/src/prompt.cxx + ${LIBRARY_DIR}/src/replxx_impl.cxx + ${LIBRARY_DIR}/src/replxx.cxx + ${LIBRARY_DIR}/src/util.cxx + ${LIBRARY_DIR}/src/wcwidth.cpp + ) -option (USE_INTERNAL_REPLXX_LIBRARY "Use internal replxx library" ${NOT_UNBUNDLED}) + add_library (replxx ${SRCS}) + target_include_directories(replxx SYSTEM PUBLIC ${LIBRARY_DIR}/include) + else () + find_library(LIBRARY_REPLXX NAMES replxx replxx-static) + find_path(INCLUDE_REPLXX replxx.hxx) -if (NOT USE_INTERNAL_REPLXX_LIBRARY) - find_library(LIBRARY_REPLXX NAMES replxx replxx-static) - find_path(INCLUDE_REPLXX replxx.hxx) + add_library(replxx UNKNOWN IMPORTED) + set_property(TARGET replxx PROPERTY IMPORTED_LOCATION ${LIBRARY_REPLXX}) + target_include_directories(replxx SYSTEM PUBLIC ${INCLUDE_REPLXX}) - if (LIBRARY_REPLXX AND INCLUDE_REPLXX) - set(CMAKE_REQUIRED_LIBRARIES ${LIBRARY_REPLXX}) - set(CMAKE_REQUIRED_INCLUDES ${INCLUDE_REPLXX}) + set(CMAKE_REQUIRED_LIBRARIES replxx) check_cxx_source_compiles( " #include <replxx.hxx> @@ -32,43 +41,20 @@ if (NOT USE_INTERNAL_REPLXX_LIBRARY) ) if (NOT EXTERNAL_REPLXX_WORKS) - message (${RECONFIGURE_MESSAGE_LEVEL} "replxx is unusable: ${LIBRARY_REPLXX} ${INCLUDE_REPLXX}") - else() - add_library(replxx UNKNOWN IMPORTED) - set_property(TARGET replxx PROPERTY IMPORTED_LOCATION ${LIBRARY_REPLXX}) - target_include_directories(replxx SYSTEM PUBLIC ${INCLUDE_REPLXX}) - endif() - else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system replxx") - endif() -endif() + message (FATAL_ERROR "replxx is unusable: ${LIBRARY_REPLXX} ${INCLUDE_REPLXX}") + endif () + endif () + if (COMPILER_CLANG) + target_compile_options(replxx PRIVATE -Wno-documentation) + endif () -if (NOT LIBRARY_REPLXX OR NOT INCLUDE_REPLXX OR NOT EXTERNAL_REPLXX_WORKS) - set(USE_INTERNAL_REPLXX_LIBRARY 1) - set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/replxx") + target_compile_definitions(replxx PUBLIC USE_REPLXX=1) - set(SRCS - ${LIBRARY_DIR}/src/conversion.cxx - ${LIBRARY_DIR}/src/ConvertUTF.cpp - ${LIBRARY_DIR}/src/escape.cxx - ${LIBRARY_DIR}/src/history.cxx - ${LIBRARY_DIR}/src/io.cxx - ${LIBRARY_DIR}/src/prompt.cxx - ${LIBRARY_DIR}/src/replxx_impl.cxx - ${LIBRARY_DIR}/src/replxx.cxx - ${LIBRARY_DIR}/src/util.cxx - ${LIBRARY_DIR}/src/wcwidth.cpp - ) + message (STATUS "Using replxx") +else () + add_library(replxx INTERFACE) + target_compile_definitions(replxx INTERFACE USE_REPLXX=0) - add_library (replxx ${SRCS}) - target_include_directories(replxx SYSTEM PUBLIC ${LIBRARY_DIR}/include) + message (STATUS "Not using replxx (Beware! Runtime fallback to readline is possible!)") endif () - -if (COMPILER_CLANG) - target_compile_options(replxx PRIVATE -Wno-documentation) -endif () - -target_compile_definitions(replxx PUBLIC USE_REPLXX=1) - -message (STATUS "Using replxx") diff --git a/contrib/unixodbc-cmake/CMakeLists.txt b/contrib/unixodbc-cmake/CMakeLists.txt index 7a585161cc1..658fa3329d3 100644 --- a/contrib/unixodbc-cmake/CMakeLists.txt +++ b/contrib/unixodbc-cmake/CMakeLists.txt @@ -1,336 +1,318 @@ option (ENABLE_ODBC "Enable ODBC library" ${ENABLE_LIBRARIES}) if (NOT OS_LINUX) - if (ENABLE_ODBC) - message(STATUS "ODBC is only supported on Linux") - endif() set (ENABLE_ODBC OFF CACHE INTERNAL "") endif () -if (NOT ENABLE_ODBC) - if (USE_INTERNAL_ODBC_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal ODBC with ENABLE_ODBC=OFF") - endif() +if (ENABLE_ODBC) + option (USE_INTERNAL_ODBC_LIBRARY "Use internal ODBC library" ${NOT_UNBUNDLED}) + if (USE_INTERNAL_ODBC_LIBRARY) + set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/unixodbc) + + # ltdl + + set (SRCS_LTDL + # This file is generated by 'libtool' inside libltdl directory and then removed. + linux_x86_64/libltdl/libltdlcS.c + + ${LIBRARY_DIR}/libltdl/lt__alloc.c + ${LIBRARY_DIR}/libltdl/lt__strl.c + ${LIBRARY_DIR}/libltdl/ltdl.c + ${LIBRARY_DIR}/libltdl/lt_dlloader.c + ${LIBRARY_DIR}/libltdl/slist.c + ${LIBRARY_DIR}/libltdl/lt_error.c + ${LIBRARY_DIR}/libltdl/loaders/dlopen.c + ${LIBRARY_DIR}/libltdl/loaders/preopen.c + ) + + add_library (ltdl ${SRCS_LTDL}) + + target_include_directories(ltdl + PRIVATE + linux_x86_64/libltdl + PUBLIC + ${LIBRARY_DIR}/libltdl + ${LIBRARY_DIR}/libltdl/libltdl + ) + target_compile_definitions(ltdl PRIVATE -DHAVE_CONFIG_H -DLTDL -DLTDLOPEN=libltdlc) + target_compile_options(ltdl PRIVATE -Wno-constant-logical-operand -Wno-unknown-warning-option -O2) + + # odbc + + set (SRCS + ${LIBRARY_DIR}/DriverManager/__attribute.c + ${LIBRARY_DIR}/DriverManager/__connection.c + ${LIBRARY_DIR}/DriverManager/__handles.c + ${LIBRARY_DIR}/DriverManager/__info.c + ${LIBRARY_DIR}/DriverManager/__stats.c + ${LIBRARY_DIR}/DriverManager/SQLAllocConnect.c + ${LIBRARY_DIR}/DriverManager/SQLAllocEnv.c + ${LIBRARY_DIR}/DriverManager/SQLAllocHandle.c + ${LIBRARY_DIR}/DriverManager/SQLAllocHandleStd.c + ${LIBRARY_DIR}/DriverManager/SQLAllocStmt.c + ${LIBRARY_DIR}/DriverManager/SQLBindCol.c + ${LIBRARY_DIR}/DriverManager/SQLBindParam.c + ${LIBRARY_DIR}/DriverManager/SQLBindParameter.c + ${LIBRARY_DIR}/DriverManager/SQLBrowseConnect.c + ${LIBRARY_DIR}/DriverManager/SQLBrowseConnectW.c + ${LIBRARY_DIR}/DriverManager/SQLBulkOperations.c + ${LIBRARY_DIR}/DriverManager/SQLCancel.c + ${LIBRARY_DIR}/DriverManager/SQLCancelHandle.c + ${LIBRARY_DIR}/DriverManager/SQLCloseCursor.c + ${LIBRARY_DIR}/DriverManager/SQLColAttribute.c + ${LIBRARY_DIR}/DriverManager/SQLColAttributes.c + ${LIBRARY_DIR}/DriverManager/SQLColAttributesW.c + ${LIBRARY_DIR}/DriverManager/SQLColAttributeW.c + ${LIBRARY_DIR}/DriverManager/SQLColumnPrivileges.c + ${LIBRARY_DIR}/DriverManager/SQLColumnPrivilegesW.c + ${LIBRARY_DIR}/DriverManager/SQLColumns.c + ${LIBRARY_DIR}/DriverManager/SQLColumnsW.c + ${LIBRARY_DIR}/DriverManager/SQLConnect.c + ${LIBRARY_DIR}/DriverManager/SQLConnectW.c + ${LIBRARY_DIR}/DriverManager/SQLCopyDesc.c + ${LIBRARY_DIR}/DriverManager/SQLDataSources.c + ${LIBRARY_DIR}/DriverManager/SQLDataSourcesW.c + ${LIBRARY_DIR}/DriverManager/SQLDescribeCol.c + ${LIBRARY_DIR}/DriverManager/SQLDescribeColW.c + ${LIBRARY_DIR}/DriverManager/SQLDescribeParam.c + ${LIBRARY_DIR}/DriverManager/SQLDisconnect.c + ${LIBRARY_DIR}/DriverManager/SQLDriverConnect.c + ${LIBRARY_DIR}/DriverManager/SQLDriverConnectW.c + ${LIBRARY_DIR}/DriverManager/SQLDrivers.c + ${LIBRARY_DIR}/DriverManager/SQLDriversW.c + ${LIBRARY_DIR}/DriverManager/SQLEndTran.c + ${LIBRARY_DIR}/DriverManager/SQLError.c + ${LIBRARY_DIR}/DriverManager/SQLErrorW.c + ${LIBRARY_DIR}/DriverManager/SQLExecDirect.c + ${LIBRARY_DIR}/DriverManager/SQLExecDirectW.c + ${LIBRARY_DIR}/DriverManager/SQLExecute.c + ${LIBRARY_DIR}/DriverManager/SQLExtendedFetch.c + ${LIBRARY_DIR}/DriverManager/SQLFetch.c + ${LIBRARY_DIR}/DriverManager/SQLFetchScroll.c + ${LIBRARY_DIR}/DriverManager/SQLForeignKeys.c + ${LIBRARY_DIR}/DriverManager/SQLForeignKeysW.c + ${LIBRARY_DIR}/DriverManager/SQLFreeConnect.c + ${LIBRARY_DIR}/DriverManager/SQLFreeEnv.c + ${LIBRARY_DIR}/DriverManager/SQLFreeHandle.c + ${LIBRARY_DIR}/DriverManager/SQLFreeStmt.c + ${LIBRARY_DIR}/DriverManager/SQLGetConnectAttr.c + ${LIBRARY_DIR}/DriverManager/SQLGetConnectAttrW.c + ${LIBRARY_DIR}/DriverManager/SQLGetConnectOption.c + ${LIBRARY_DIR}/DriverManager/SQLGetConnectOptionW.c + ${LIBRARY_DIR}/DriverManager/SQLGetCursorName.c + ${LIBRARY_DIR}/DriverManager/SQLGetCursorNameW.c + ${LIBRARY_DIR}/DriverManager/SQLGetData.c + ${LIBRARY_DIR}/DriverManager/SQLGetDescField.c + ${LIBRARY_DIR}/DriverManager/SQLGetDescFieldW.c + ${LIBRARY_DIR}/DriverManager/SQLGetDescRec.c + ${LIBRARY_DIR}/DriverManager/SQLGetDescRecW.c + ${LIBRARY_DIR}/DriverManager/SQLGetDiagField.c + ${LIBRARY_DIR}/DriverManager/SQLGetDiagFieldW.c + ${LIBRARY_DIR}/DriverManager/SQLGetDiagRec.c + ${LIBRARY_DIR}/DriverManager/SQLGetDiagRecW.c + ${LIBRARY_DIR}/DriverManager/SQLGetEnvAttr.c + ${LIBRARY_DIR}/DriverManager/SQLGetFunctions.c + ${LIBRARY_DIR}/DriverManager/SQLGetInfo.c + ${LIBRARY_DIR}/DriverManager/SQLGetInfoW.c + ${LIBRARY_DIR}/DriverManager/SQLGetStmtAttr.c + ${LIBRARY_DIR}/DriverManager/SQLGetStmtAttrW.c + ${LIBRARY_DIR}/DriverManager/SQLGetStmtOption.c + ${LIBRARY_DIR}/DriverManager/SQLGetTypeInfo.c + ${LIBRARY_DIR}/DriverManager/SQLGetTypeInfoW.c + ${LIBRARY_DIR}/DriverManager/SQLMoreResults.c + ${LIBRARY_DIR}/DriverManager/SQLNativeSql.c + ${LIBRARY_DIR}/DriverManager/SQLNativeSqlW.c + ${LIBRARY_DIR}/DriverManager/SQLNumParams.c + ${LIBRARY_DIR}/DriverManager/SQLNumResultCols.c + ${LIBRARY_DIR}/DriverManager/SQLParamData.c + ${LIBRARY_DIR}/DriverManager/SQLParamOptions.c + ${LIBRARY_DIR}/DriverManager/SQLPrepare.c + ${LIBRARY_DIR}/DriverManager/SQLPrepareW.c + ${LIBRARY_DIR}/DriverManager/SQLPrimaryKeys.c + ${LIBRARY_DIR}/DriverManager/SQLPrimaryKeysW.c + ${LIBRARY_DIR}/DriverManager/SQLProcedureColumns.c + ${LIBRARY_DIR}/DriverManager/SQLProcedureColumnsW.c + ${LIBRARY_DIR}/DriverManager/SQLProcedures.c + ${LIBRARY_DIR}/DriverManager/SQLProceduresW.c + ${LIBRARY_DIR}/DriverManager/SQLPutData.c + ${LIBRARY_DIR}/DriverManager/SQLRowCount.c + ${LIBRARY_DIR}/DriverManager/SQLSetConnectAttr.c + ${LIBRARY_DIR}/DriverManager/SQLSetConnectAttrW.c + ${LIBRARY_DIR}/DriverManager/SQLSetConnectOption.c + ${LIBRARY_DIR}/DriverManager/SQLSetConnectOptionW.c + ${LIBRARY_DIR}/DriverManager/SQLSetCursorName.c + ${LIBRARY_DIR}/DriverManager/SQLSetCursorNameW.c + ${LIBRARY_DIR}/DriverManager/SQLSetDescField.c + ${LIBRARY_DIR}/DriverManager/SQLSetDescFieldW.c + ${LIBRARY_DIR}/DriverManager/SQLSetDescRec.c + ${LIBRARY_DIR}/DriverManager/SQLSetEnvAttr.c + ${LIBRARY_DIR}/DriverManager/SQLSetParam.c + ${LIBRARY_DIR}/DriverManager/SQLSetPos.c + ${LIBRARY_DIR}/DriverManager/SQLSetScrollOptions.c + ${LIBRARY_DIR}/DriverManager/SQLSetStmtAttr.c + ${LIBRARY_DIR}/DriverManager/SQLSetStmtAttrW.c + ${LIBRARY_DIR}/DriverManager/SQLSetStmtOption.c + ${LIBRARY_DIR}/DriverManager/SQLSetStmtOptionW.c + ${LIBRARY_DIR}/DriverManager/SQLSpecialColumns.c + ${LIBRARY_DIR}/DriverManager/SQLSpecialColumnsW.c + ${LIBRARY_DIR}/DriverManager/SQLStatistics.c + ${LIBRARY_DIR}/DriverManager/SQLStatisticsW.c + ${LIBRARY_DIR}/DriverManager/SQLTablePrivileges.c + ${LIBRARY_DIR}/DriverManager/SQLTablePrivilegesW.c + ${LIBRARY_DIR}/DriverManager/SQLTables.c + ${LIBRARY_DIR}/DriverManager/SQLTablesW.c + ${LIBRARY_DIR}/DriverManager/SQLTransact.c + ${LIBRARY_DIR}/ini/_iniDump.c + ${LIBRARY_DIR}/ini/_iniObjectRead.c + ${LIBRARY_DIR}/ini/_iniPropertyRead.c + ${LIBRARY_DIR}/ini/_iniScanUntilObject.c + ${LIBRARY_DIR}/ini/iniAllTrim.c + ${LIBRARY_DIR}/ini/iniAppend.c + ${LIBRARY_DIR}/ini/iniClose.c + ${LIBRARY_DIR}/ini/iniCommit.c + ${LIBRARY_DIR}/ini/iniCursor.c + ${LIBRARY_DIR}/ini/iniDelete.c + ${LIBRARY_DIR}/ini/iniElement.c + ${LIBRARY_DIR}/ini/iniElementCount.c + ${LIBRARY_DIR}/ini/iniGetBookmark.c + ${LIBRARY_DIR}/ini/iniGotoBookmark.c + ${LIBRARY_DIR}/ini/iniObject.c + ${LIBRARY_DIR}/ini/iniObjectDelete.c + ${LIBRARY_DIR}/ini/iniObjectEOL.c + ${LIBRARY_DIR}/ini/iniObjectFirst.c + ${LIBRARY_DIR}/ini/iniObjectInsert.c + ${LIBRARY_DIR}/ini/iniObjectLast.c + ${LIBRARY_DIR}/ini/iniObjectNext.c + ${LIBRARY_DIR}/ini/iniObjectSeek.c + ${LIBRARY_DIR}/ini/iniObjectSeekSure.c + ${LIBRARY_DIR}/ini/iniObjectUpdate.c + ${LIBRARY_DIR}/ini/iniOpen.c + ${LIBRARY_DIR}/ini/iniProperty.c + ${LIBRARY_DIR}/ini/iniPropertyDelete.c + ${LIBRARY_DIR}/ini/iniPropertyEOL.c + ${LIBRARY_DIR}/ini/iniPropertyFirst.c + ${LIBRARY_DIR}/ini/iniPropertyInsert.c + ${LIBRARY_DIR}/ini/iniPropertyLast.c + ${LIBRARY_DIR}/ini/iniPropertyNext.c + ${LIBRARY_DIR}/ini/iniPropertySeek.c + ${LIBRARY_DIR}/ini/iniPropertySeekSure.c + ${LIBRARY_DIR}/ini/iniPropertyUpdate.c + ${LIBRARY_DIR}/ini/iniPropertyValue.c + ${LIBRARY_DIR}/ini/iniToUpper.c + ${LIBRARY_DIR}/ini/iniValue.c + ${LIBRARY_DIR}/log/_logFreeMsg.c + ${LIBRARY_DIR}/log/logClear.c + ${LIBRARY_DIR}/log/logClose.c + ${LIBRARY_DIR}/log/logOn.c + ${LIBRARY_DIR}/log/logOpen.c + ${LIBRARY_DIR}/log/logPeekMsg.c + ${LIBRARY_DIR}/log/logPopMsg.c + ${LIBRARY_DIR}/log/logPushMsg.c + ${LIBRARY_DIR}/lst/_lstAdjustCurrent.c + ${LIBRARY_DIR}/lst/_lstDump.c + ${LIBRARY_DIR}/lst/_lstFreeItem.c + ${LIBRARY_DIR}/lst/_lstNextValidItem.c + ${LIBRARY_DIR}/lst/_lstPrevValidItem.c + ${LIBRARY_DIR}/lst/_lstVisible.c + ${LIBRARY_DIR}/lst/lstAppend.c + ${LIBRARY_DIR}/lst/lstClose.c + ${LIBRARY_DIR}/lst/lstDelete.c + ${LIBRARY_DIR}/lst/lstEOL.c + ${LIBRARY_DIR}/lst/lstFirst.c + ${LIBRARY_DIR}/lst/lstGet.c + ${LIBRARY_DIR}/lst/lstGetBookMark.c + ${LIBRARY_DIR}/lst/lstGoto.c + ${LIBRARY_DIR}/lst/lstGotoBookMark.c + ${LIBRARY_DIR}/lst/lstInsert.c + ${LIBRARY_DIR}/lst/lstLast.c + ${LIBRARY_DIR}/lst/lstNext.c + ${LIBRARY_DIR}/lst/lstOpen.c + ${LIBRARY_DIR}/lst/lstOpenCursor.c + ${LIBRARY_DIR}/lst/lstPrev.c + ${LIBRARY_DIR}/lst/lstSeek.c + ${LIBRARY_DIR}/lst/lstSeekItem.c + ${LIBRARY_DIR}/lst/lstSet.c + ${LIBRARY_DIR}/lst/lstSetFreeFunc.c + ${LIBRARY_DIR}/odbcinst/_logging.c + ${LIBRARY_DIR}/odbcinst/_odbcinst_ConfigModeINI.c + ${LIBRARY_DIR}/odbcinst/_odbcinst_GetEntries.c + ${LIBRARY_DIR}/odbcinst/_odbcinst_GetSections.c + ${LIBRARY_DIR}/odbcinst/_odbcinst_SystemINI.c + ${LIBRARY_DIR}/odbcinst/_odbcinst_UserINI.c + ${LIBRARY_DIR}/odbcinst/_SQLDriverConnectPrompt.c + ${LIBRARY_DIR}/odbcinst/_SQLGetInstalledDrivers.c + ${LIBRARY_DIR}/odbcinst/_SQLWriteInstalledDrivers.c + ${LIBRARY_DIR}/odbcinst/ODBCINSTConstructProperties.c + ${LIBRARY_DIR}/odbcinst/ODBCINSTDestructProperties.c + ${LIBRARY_DIR}/odbcinst/ODBCINSTSetProperty.c + ${LIBRARY_DIR}/odbcinst/ODBCINSTValidateProperties.c + ${LIBRARY_DIR}/odbcinst/ODBCINSTValidateProperty.c + ${LIBRARY_DIR}/odbcinst/SQLConfigDataSource.c + ${LIBRARY_DIR}/odbcinst/SQLConfigDriver.c + ${LIBRARY_DIR}/odbcinst/SQLCreateDataSource.c + ${LIBRARY_DIR}/odbcinst/SQLGetAvailableDrivers.c + ${LIBRARY_DIR}/odbcinst/SQLGetConfigMode.c + ${LIBRARY_DIR}/odbcinst/SQLGetInstalledDrivers.c + ${LIBRARY_DIR}/odbcinst/SQLGetPrivateProfileString.c + ${LIBRARY_DIR}/odbcinst/SQLGetTranslator.c + ${LIBRARY_DIR}/odbcinst/SQLInstallDriverEx.c + ${LIBRARY_DIR}/odbcinst/SQLInstallDriverManager.c + ${LIBRARY_DIR}/odbcinst/SQLInstallerError.c + ${LIBRARY_DIR}/odbcinst/SQLInstallODBC.c + ${LIBRARY_DIR}/odbcinst/SQLInstallTranslatorEx.c + ${LIBRARY_DIR}/odbcinst/SQLManageDataSources.c + ${LIBRARY_DIR}/odbcinst/SQLPostInstallerError.c + ${LIBRARY_DIR}/odbcinst/SQLReadFileDSN.c + ${LIBRARY_DIR}/odbcinst/SQLRemoveDriver.c + ${LIBRARY_DIR}/odbcinst/SQLRemoveDriverManager.c + ${LIBRARY_DIR}/odbcinst/SQLRemoveDSNFromIni.c + ${LIBRARY_DIR}/odbcinst/SQLRemoveTranslator.c + ${LIBRARY_DIR}/odbcinst/SQLSetConfigMode.c + ${LIBRARY_DIR}/odbcinst/SQLValidDSN.c + ${LIBRARY_DIR}/odbcinst/SQLWriteDSNToIni.c + ${LIBRARY_DIR}/odbcinst/SQLWriteFileDSN.c + ${LIBRARY_DIR}/odbcinst/SQLWritePrivateProfileString.c + ) + + add_library (unixodbc ${SRCS}) + + target_link_libraries (unixodbc PRIVATE ltdl) + + # SYSTEM_FILE_PATH was changed to /etc + + target_include_directories (unixodbc + PRIVATE + linux_x86_64/private + PUBLIC + linux_x86_64 + ${LIBRARY_DIR}/include + ) + target_compile_definitions (unixodbc PRIVATE -DHAVE_CONFIG_H) + target_compile_options (unixodbc + PRIVATE + -Wno-dangling-else + -Wno-parentheses + -Wno-misleading-indentation + -Wno-unknown-warning-option + -Wno-reserved-id-macro + -O2 + ) + else () + add_library (unixodbc UNKNOWN IMPORTED) + + find_library (LIBRARY_ODBC unixodbc) + find_path (INCLUDE_ODBC sql.h) + set_target_properties (unixodbc PROPERTIES IMPORTED_LOCATION ${LIBRARY_ODBC}) + set_target_properties (unixodbc PROPERTIES INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_ODBC}) + endif () + + target_compile_definitions (unixodbc INTERFACE USE_ODBC=1) + + message (STATUS "Using unixodbc") +else () add_library (unixodbc INTERFACE) target_compile_definitions (unixodbc INTERFACE USE_ODBC=0) message (STATUS "Not using unixodbc") - return() -endif() - -option (USE_INTERNAL_ODBC_LIBRARY "Use internal ODBC library" ${NOT_UNBUNDLED}) - -if (NOT USE_INTERNAL_ODBC_LIBRARY) - find_library (LIBRARY_ODBC unixodbc) - find_path (INCLUDE_ODBC sql.h) - - if(LIBRARY_ODBC AND INCLUDE_ODBC) - add_library (unixodbc UNKNOWN IMPORTED) - set_target_properties (unixodbc PROPERTIES IMPORTED_LOCATION ${LIBRARY_ODBC}) - set_target_properties (unixodbc PROPERTIES INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_ODBC}) - set(EXTERNAL_ODBC_LIBRARY_FOUND 1) - else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system ODBC library") - set(EXTERNAL_ODBC_LIBRARY_FOUND 0) - endif() -endif() - -if (NOT EXTERNAL_ODBC_LIBRARY_FOUND) - set (USE_INTERNAL_ODBC_LIBRARY 1) - - set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/unixodbc) - - # ltdl - - set (SRCS_LTDL - # This file is generated by 'libtool' inside libltdl directory and then removed. - linux_x86_64/libltdl/libltdlcS.c - - ${LIBRARY_DIR}/libltdl/lt__alloc.c - ${LIBRARY_DIR}/libltdl/lt__strl.c - ${LIBRARY_DIR}/libltdl/ltdl.c - ${LIBRARY_DIR}/libltdl/lt_dlloader.c - ${LIBRARY_DIR}/libltdl/slist.c - ${LIBRARY_DIR}/libltdl/lt_error.c - ${LIBRARY_DIR}/libltdl/loaders/dlopen.c - ${LIBRARY_DIR}/libltdl/loaders/preopen.c - ) - - add_library (ltdl ${SRCS_LTDL}) - - target_include_directories(ltdl - PRIVATE - linux_x86_64/libltdl - PUBLIC - ${LIBRARY_DIR}/libltdl - ${LIBRARY_DIR}/libltdl/libltdl - ) - target_compile_definitions(ltdl PRIVATE -DHAVE_CONFIG_H -DLTDL -DLTDLOPEN=libltdlc) - target_compile_options(ltdl PRIVATE -Wno-constant-logical-operand -Wno-unknown-warning-option -O2) - - # odbc - - set (SRCS - ${LIBRARY_DIR}/DriverManager/__attribute.c - ${LIBRARY_DIR}/DriverManager/__connection.c - ${LIBRARY_DIR}/DriverManager/__handles.c - ${LIBRARY_DIR}/DriverManager/__info.c - ${LIBRARY_DIR}/DriverManager/__stats.c - ${LIBRARY_DIR}/DriverManager/SQLAllocConnect.c - ${LIBRARY_DIR}/DriverManager/SQLAllocEnv.c - ${LIBRARY_DIR}/DriverManager/SQLAllocHandle.c - ${LIBRARY_DIR}/DriverManager/SQLAllocHandleStd.c - ${LIBRARY_DIR}/DriverManager/SQLAllocStmt.c - ${LIBRARY_DIR}/DriverManager/SQLBindCol.c - ${LIBRARY_DIR}/DriverManager/SQLBindParam.c - ${LIBRARY_DIR}/DriverManager/SQLBindParameter.c - ${LIBRARY_DIR}/DriverManager/SQLBrowseConnect.c - ${LIBRARY_DIR}/DriverManager/SQLBrowseConnectW.c - ${LIBRARY_DIR}/DriverManager/SQLBulkOperations.c - ${LIBRARY_DIR}/DriverManager/SQLCancel.c - ${LIBRARY_DIR}/DriverManager/SQLCancelHandle.c - ${LIBRARY_DIR}/DriverManager/SQLCloseCursor.c - ${LIBRARY_DIR}/DriverManager/SQLColAttribute.c - ${LIBRARY_DIR}/DriverManager/SQLColAttributes.c - ${LIBRARY_DIR}/DriverManager/SQLColAttributesW.c - ${LIBRARY_DIR}/DriverManager/SQLColAttributeW.c - ${LIBRARY_DIR}/DriverManager/SQLColumnPrivileges.c - ${LIBRARY_DIR}/DriverManager/SQLColumnPrivilegesW.c - ${LIBRARY_DIR}/DriverManager/SQLColumns.c - ${LIBRARY_DIR}/DriverManager/SQLColumnsW.c - ${LIBRARY_DIR}/DriverManager/SQLConnect.c - ${LIBRARY_DIR}/DriverManager/SQLConnectW.c - ${LIBRARY_DIR}/DriverManager/SQLCopyDesc.c - ${LIBRARY_DIR}/DriverManager/SQLDataSources.c - ${LIBRARY_DIR}/DriverManager/SQLDataSourcesW.c - ${LIBRARY_DIR}/DriverManager/SQLDescribeCol.c - ${LIBRARY_DIR}/DriverManager/SQLDescribeColW.c - ${LIBRARY_DIR}/DriverManager/SQLDescribeParam.c - ${LIBRARY_DIR}/DriverManager/SQLDisconnect.c - ${LIBRARY_DIR}/DriverManager/SQLDriverConnect.c - ${LIBRARY_DIR}/DriverManager/SQLDriverConnectW.c - ${LIBRARY_DIR}/DriverManager/SQLDrivers.c - ${LIBRARY_DIR}/DriverManager/SQLDriversW.c - ${LIBRARY_DIR}/DriverManager/SQLEndTran.c - ${LIBRARY_DIR}/DriverManager/SQLError.c - ${LIBRARY_DIR}/DriverManager/SQLErrorW.c - ${LIBRARY_DIR}/DriverManager/SQLExecDirect.c - ${LIBRARY_DIR}/DriverManager/SQLExecDirectW.c - ${LIBRARY_DIR}/DriverManager/SQLExecute.c - ${LIBRARY_DIR}/DriverManager/SQLExtendedFetch.c - ${LIBRARY_DIR}/DriverManager/SQLFetch.c - ${LIBRARY_DIR}/DriverManager/SQLFetchScroll.c - ${LIBRARY_DIR}/DriverManager/SQLForeignKeys.c - ${LIBRARY_DIR}/DriverManager/SQLForeignKeysW.c - ${LIBRARY_DIR}/DriverManager/SQLFreeConnect.c - ${LIBRARY_DIR}/DriverManager/SQLFreeEnv.c - ${LIBRARY_DIR}/DriverManager/SQLFreeHandle.c - ${LIBRARY_DIR}/DriverManager/SQLFreeStmt.c - ${LIBRARY_DIR}/DriverManager/SQLGetConnectAttr.c - ${LIBRARY_DIR}/DriverManager/SQLGetConnectAttrW.c - ${LIBRARY_DIR}/DriverManager/SQLGetConnectOption.c - ${LIBRARY_DIR}/DriverManager/SQLGetConnectOptionW.c - ${LIBRARY_DIR}/DriverManager/SQLGetCursorName.c - ${LIBRARY_DIR}/DriverManager/SQLGetCursorNameW.c - ${LIBRARY_DIR}/DriverManager/SQLGetData.c - ${LIBRARY_DIR}/DriverManager/SQLGetDescField.c - ${LIBRARY_DIR}/DriverManager/SQLGetDescFieldW.c - ${LIBRARY_DIR}/DriverManager/SQLGetDescRec.c - ${LIBRARY_DIR}/DriverManager/SQLGetDescRecW.c - ${LIBRARY_DIR}/DriverManager/SQLGetDiagField.c - ${LIBRARY_DIR}/DriverManager/SQLGetDiagFieldW.c - ${LIBRARY_DIR}/DriverManager/SQLGetDiagRec.c - ${LIBRARY_DIR}/DriverManager/SQLGetDiagRecW.c - ${LIBRARY_DIR}/DriverManager/SQLGetEnvAttr.c - ${LIBRARY_DIR}/DriverManager/SQLGetFunctions.c - ${LIBRARY_DIR}/DriverManager/SQLGetInfo.c - ${LIBRARY_DIR}/DriverManager/SQLGetInfoW.c - ${LIBRARY_DIR}/DriverManager/SQLGetStmtAttr.c - ${LIBRARY_DIR}/DriverManager/SQLGetStmtAttrW.c - ${LIBRARY_DIR}/DriverManager/SQLGetStmtOption.c - ${LIBRARY_DIR}/DriverManager/SQLGetTypeInfo.c - ${LIBRARY_DIR}/DriverManager/SQLGetTypeInfoW.c - ${LIBRARY_DIR}/DriverManager/SQLMoreResults.c - ${LIBRARY_DIR}/DriverManager/SQLNativeSql.c - ${LIBRARY_DIR}/DriverManager/SQLNativeSqlW.c - ${LIBRARY_DIR}/DriverManager/SQLNumParams.c - ${LIBRARY_DIR}/DriverManager/SQLNumResultCols.c - ${LIBRARY_DIR}/DriverManager/SQLParamData.c - ${LIBRARY_DIR}/DriverManager/SQLParamOptions.c - ${LIBRARY_DIR}/DriverManager/SQLPrepare.c - ${LIBRARY_DIR}/DriverManager/SQLPrepareW.c - ${LIBRARY_DIR}/DriverManager/SQLPrimaryKeys.c - ${LIBRARY_DIR}/DriverManager/SQLPrimaryKeysW.c - ${LIBRARY_DIR}/DriverManager/SQLProcedureColumns.c - ${LIBRARY_DIR}/DriverManager/SQLProcedureColumnsW.c - ${LIBRARY_DIR}/DriverManager/SQLProcedures.c - ${LIBRARY_DIR}/DriverManager/SQLProceduresW.c - ${LIBRARY_DIR}/DriverManager/SQLPutData.c - ${LIBRARY_DIR}/DriverManager/SQLRowCount.c - ${LIBRARY_DIR}/DriverManager/SQLSetConnectAttr.c - ${LIBRARY_DIR}/DriverManager/SQLSetConnectAttrW.c - ${LIBRARY_DIR}/DriverManager/SQLSetConnectOption.c - ${LIBRARY_DIR}/DriverManager/SQLSetConnectOptionW.c - ${LIBRARY_DIR}/DriverManager/SQLSetCursorName.c - ${LIBRARY_DIR}/DriverManager/SQLSetCursorNameW.c - ${LIBRARY_DIR}/DriverManager/SQLSetDescField.c - ${LIBRARY_DIR}/DriverManager/SQLSetDescFieldW.c - ${LIBRARY_DIR}/DriverManager/SQLSetDescRec.c - ${LIBRARY_DIR}/DriverManager/SQLSetEnvAttr.c - ${LIBRARY_DIR}/DriverManager/SQLSetParam.c - ${LIBRARY_DIR}/DriverManager/SQLSetPos.c - ${LIBRARY_DIR}/DriverManager/SQLSetScrollOptions.c - ${LIBRARY_DIR}/DriverManager/SQLSetStmtAttr.c - ${LIBRARY_DIR}/DriverManager/SQLSetStmtAttrW.c - ${LIBRARY_DIR}/DriverManager/SQLSetStmtOption.c - ${LIBRARY_DIR}/DriverManager/SQLSetStmtOptionW.c - ${LIBRARY_DIR}/DriverManager/SQLSpecialColumns.c - ${LIBRARY_DIR}/DriverManager/SQLSpecialColumnsW.c - ${LIBRARY_DIR}/DriverManager/SQLStatistics.c - ${LIBRARY_DIR}/DriverManager/SQLStatisticsW.c - ${LIBRARY_DIR}/DriverManager/SQLTablePrivileges.c - ${LIBRARY_DIR}/DriverManager/SQLTablePrivilegesW.c - ${LIBRARY_DIR}/DriverManager/SQLTables.c - ${LIBRARY_DIR}/DriverManager/SQLTablesW.c - ${LIBRARY_DIR}/DriverManager/SQLTransact.c - ${LIBRARY_DIR}/ini/_iniDump.c - ${LIBRARY_DIR}/ini/_iniObjectRead.c - ${LIBRARY_DIR}/ini/_iniPropertyRead.c - ${LIBRARY_DIR}/ini/_iniScanUntilObject.c - ${LIBRARY_DIR}/ini/iniAllTrim.c - ${LIBRARY_DIR}/ini/iniAppend.c - ${LIBRARY_DIR}/ini/iniClose.c - ${LIBRARY_DIR}/ini/iniCommit.c - ${LIBRARY_DIR}/ini/iniCursor.c - ${LIBRARY_DIR}/ini/iniDelete.c - ${LIBRARY_DIR}/ini/iniElement.c - ${LIBRARY_DIR}/ini/iniElementCount.c - ${LIBRARY_DIR}/ini/iniGetBookmark.c - ${LIBRARY_DIR}/ini/iniGotoBookmark.c - ${LIBRARY_DIR}/ini/iniObject.c - ${LIBRARY_DIR}/ini/iniObjectDelete.c - ${LIBRARY_DIR}/ini/iniObjectEOL.c - ${LIBRARY_DIR}/ini/iniObjectFirst.c - ${LIBRARY_DIR}/ini/iniObjectInsert.c - ${LIBRARY_DIR}/ini/iniObjectLast.c - ${LIBRARY_DIR}/ini/iniObjectNext.c - ${LIBRARY_DIR}/ini/iniObjectSeek.c - ${LIBRARY_DIR}/ini/iniObjectSeekSure.c - ${LIBRARY_DIR}/ini/iniObjectUpdate.c - ${LIBRARY_DIR}/ini/iniOpen.c - ${LIBRARY_DIR}/ini/iniProperty.c - ${LIBRARY_DIR}/ini/iniPropertyDelete.c - ${LIBRARY_DIR}/ini/iniPropertyEOL.c - ${LIBRARY_DIR}/ini/iniPropertyFirst.c - ${LIBRARY_DIR}/ini/iniPropertyInsert.c - ${LIBRARY_DIR}/ini/iniPropertyLast.c - ${LIBRARY_DIR}/ini/iniPropertyNext.c - ${LIBRARY_DIR}/ini/iniPropertySeek.c - ${LIBRARY_DIR}/ini/iniPropertySeekSure.c - ${LIBRARY_DIR}/ini/iniPropertyUpdate.c - ${LIBRARY_DIR}/ini/iniPropertyValue.c - ${LIBRARY_DIR}/ini/iniToUpper.c - ${LIBRARY_DIR}/ini/iniValue.c - ${LIBRARY_DIR}/log/_logFreeMsg.c - ${LIBRARY_DIR}/log/logClear.c - ${LIBRARY_DIR}/log/logClose.c - ${LIBRARY_DIR}/log/logOn.c - ${LIBRARY_DIR}/log/logOpen.c - ${LIBRARY_DIR}/log/logPeekMsg.c - ${LIBRARY_DIR}/log/logPopMsg.c - ${LIBRARY_DIR}/log/logPushMsg.c - ${LIBRARY_DIR}/lst/_lstAdjustCurrent.c - ${LIBRARY_DIR}/lst/_lstDump.c - ${LIBRARY_DIR}/lst/_lstFreeItem.c - ${LIBRARY_DIR}/lst/_lstNextValidItem.c - ${LIBRARY_DIR}/lst/_lstPrevValidItem.c - ${LIBRARY_DIR}/lst/_lstVisible.c - ${LIBRARY_DIR}/lst/lstAppend.c - ${LIBRARY_DIR}/lst/lstClose.c - ${LIBRARY_DIR}/lst/lstDelete.c - ${LIBRARY_DIR}/lst/lstEOL.c - ${LIBRARY_DIR}/lst/lstFirst.c - ${LIBRARY_DIR}/lst/lstGet.c - ${LIBRARY_DIR}/lst/lstGetBookMark.c - ${LIBRARY_DIR}/lst/lstGoto.c - ${LIBRARY_DIR}/lst/lstGotoBookMark.c - ${LIBRARY_DIR}/lst/lstInsert.c - ${LIBRARY_DIR}/lst/lstLast.c - ${LIBRARY_DIR}/lst/lstNext.c - ${LIBRARY_DIR}/lst/lstOpen.c - ${LIBRARY_DIR}/lst/lstOpenCursor.c - ${LIBRARY_DIR}/lst/lstPrev.c - ${LIBRARY_DIR}/lst/lstSeek.c - ${LIBRARY_DIR}/lst/lstSeekItem.c - ${LIBRARY_DIR}/lst/lstSet.c - ${LIBRARY_DIR}/lst/lstSetFreeFunc.c - ${LIBRARY_DIR}/odbcinst/_logging.c - ${LIBRARY_DIR}/odbcinst/_odbcinst_ConfigModeINI.c - ${LIBRARY_DIR}/odbcinst/_odbcinst_GetEntries.c - ${LIBRARY_DIR}/odbcinst/_odbcinst_GetSections.c - ${LIBRARY_DIR}/odbcinst/_odbcinst_SystemINI.c - ${LIBRARY_DIR}/odbcinst/_odbcinst_UserINI.c - ${LIBRARY_DIR}/odbcinst/_SQLDriverConnectPrompt.c - ${LIBRARY_DIR}/odbcinst/_SQLGetInstalledDrivers.c - ${LIBRARY_DIR}/odbcinst/_SQLWriteInstalledDrivers.c - ${LIBRARY_DIR}/odbcinst/ODBCINSTConstructProperties.c - ${LIBRARY_DIR}/odbcinst/ODBCINSTDestructProperties.c - ${LIBRARY_DIR}/odbcinst/ODBCINSTSetProperty.c - ${LIBRARY_DIR}/odbcinst/ODBCINSTValidateProperties.c - ${LIBRARY_DIR}/odbcinst/ODBCINSTValidateProperty.c - ${LIBRARY_DIR}/odbcinst/SQLConfigDataSource.c - ${LIBRARY_DIR}/odbcinst/SQLConfigDriver.c - ${LIBRARY_DIR}/odbcinst/SQLCreateDataSource.c - ${LIBRARY_DIR}/odbcinst/SQLGetAvailableDrivers.c - ${LIBRARY_DIR}/odbcinst/SQLGetConfigMode.c - ${LIBRARY_DIR}/odbcinst/SQLGetInstalledDrivers.c - ${LIBRARY_DIR}/odbcinst/SQLGetPrivateProfileString.c - ${LIBRARY_DIR}/odbcinst/SQLGetTranslator.c - ${LIBRARY_DIR}/odbcinst/SQLInstallDriverEx.c - ${LIBRARY_DIR}/odbcinst/SQLInstallDriverManager.c - ${LIBRARY_DIR}/odbcinst/SQLInstallerError.c - ${LIBRARY_DIR}/odbcinst/SQLInstallODBC.c - ${LIBRARY_DIR}/odbcinst/SQLInstallTranslatorEx.c - ${LIBRARY_DIR}/odbcinst/SQLManageDataSources.c - ${LIBRARY_DIR}/odbcinst/SQLPostInstallerError.c - ${LIBRARY_DIR}/odbcinst/SQLReadFileDSN.c - ${LIBRARY_DIR}/odbcinst/SQLRemoveDriver.c - ${LIBRARY_DIR}/odbcinst/SQLRemoveDriverManager.c - ${LIBRARY_DIR}/odbcinst/SQLRemoveDSNFromIni.c - ${LIBRARY_DIR}/odbcinst/SQLRemoveTranslator.c - ${LIBRARY_DIR}/odbcinst/SQLSetConfigMode.c - ${LIBRARY_DIR}/odbcinst/SQLValidDSN.c - ${LIBRARY_DIR}/odbcinst/SQLWriteDSNToIni.c - ${LIBRARY_DIR}/odbcinst/SQLWriteFileDSN.c - ${LIBRARY_DIR}/odbcinst/SQLWritePrivateProfileString.c - ) - - add_library (unixodbc ${SRCS}) - - target_link_libraries (unixodbc PRIVATE ltdl) - - # SYSTEM_FILE_PATH was changed to /etc - - target_include_directories (unixodbc - PRIVATE - linux_x86_64/private - PUBLIC - linux_x86_64 - ${LIBRARY_DIR}/include - ) - target_compile_definitions (unixodbc PRIVATE -DHAVE_CONFIG_H) - target_compile_options (unixodbc - PRIVATE - -Wno-dangling-else - -Wno-parentheses - -Wno-misleading-indentation - -Wno-unknown-warning-option - -Wno-reserved-id-macro - -O2 - ) endif () - -target_compile_definitions (unixodbc INTERFACE USE_ODBC=1) - -message (STATUS "Using unixodbc") From b7ba0b64dca342c10101c777f327dd5dceef49f1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin <a3at.mail@gmail.com> Date: Thu, 13 Aug 2020 22:51:41 +0300 Subject: [PATCH 311/374] Reset signal handlers just before closing signal pipe to avoid EBADF after stack trace: (gdb) bt 0 0x00007ffff7dda355 in raise () from /usr/lib/libc.so.6 1 0x00007ffff7dc3853 in abort () from /usr/lib/libc.so.6 2 0x0000000013a78180 in abort_message (format=format@entry=0x5781e10 "terminate_handler unexpectedly threw an exception") at abort_message.cpp:76 3 0x0000000013a899ff in std::__terminate (func=<optimized out>) at cxa_handlers.cpp:67 4 0x0000000013a771a1 in __cxxabiv1::call_terminate (native_exception=native_exception@entry=true, unwind_exception=unwind_exception@entry=0x7ffff70e9060) at cxa_personality.cpp:323 5 0x0000000013a77669 in __cxxabiv1::scan_eh_tab (results=..., actions=<optimized out>, native_exception=native_exception@entry=true, unwind_exception=0x7ffff70e9060, context=context@entry=0x7fffffffc3a0) at cxa_personality.cpp:887 6 0x0000000013a77bde in __cxxabiv1::__gxx_personality_v0 (version=<optimized out>, actions=<optimized out>, exceptionClass=<optimized out>, unwind_exception=0x7ffff70e9060, context=0x7fffffffc3a0) at cxa_personality.cpp:969 7 0x0000000013a962d7 in unwind_phase1 (exception_object=0x7ffff70e9060, cursor=0x7fffffffc3a0, uc=0x7fffffffc2f0) at UnwindLevel1.c:98 8 _Unwind_RaiseException (exception_object=exception_object@entry=0x7ffff70e9060) at UnwindLevel1.c:363 9 0x0000000013a8929b in __cxxabiv1::__cxa_rethrow () at cxa_exception.cpp:607 10 0x000000000f170ad2 in DB::WriteBuffer::next (this=0x7fffffffc510) at WriteBuffer.h:52 11 writeSignalIDtoSignalPipe (sig=<optimized out>) at BaseDaemon.cpp:111 12 <signal handler called> 13 0x00007ffff7e9895b in munmap () from /usr/lib/libc.so.6 14 0x00007ffff7f7e255 in free_stacks () from /usr/lib/libpthread.so.0 15 0x00007ffff7f7e6aa in __deallocate_stack () from /usr/lib/libpthread.so.0 16 0x00007ffff7f80898 in __pthread_clockjoin_ex () from /usr/lib/libpthread.so.0 17 0x0000000013a759f7 in std::__1::__libcpp_thread_join (__t=0x7fff0aa2b090) at __threading_support:409 18 std::__1::thread::join (this=this@entry=0x7fff0aa2b090) at thread.cpp:56 19 0x000000000a4c9399 in ThreadPoolImpl<std::__1::thread>::finalize (this=this@entry=0x7ffff7053900) at list:355 20 0x000000000a4c9426 in ThreadPoolImpl<std::__1::thread>::~ThreadPoolImpl (this=0x7ffff7053900, __in_chrg=<optimized out>) at ThreadPool.cpp:172 21 0x000000000a4c8a9c in GlobalThreadPool::~GlobalThreadPool (this=0x7ffff7053900, __in_chrg=<optimized out>) at ThreadPool.h:132 22 std::__1::default_delete<GlobalThreadPool>::operator() (__ptr=0x7ffff7053900, this=<optimized out>) at memory:2363 23 std::__1::unique_ptr<GlobalThreadPool, std::__1::default_delete<GlobalThreadPool> >::reset (__p=0x0, this=<optimized out>) at memory:2618 24 std::__1::unique_ptr<GlobalThreadPool, std::__1::default_delete<GlobalThreadPool> >::~unique_ptr (this=<optimized out>, __in_chrg=<optimized out>) at memory:2572 25 0x00007ffff7ddcc57 in __run_exit_handlers () from /usr/lib/libc.so.6 26 0x00007ffff7ddcdfe in exit () from /usr/lib/libc.so.6 27 0x00007ffff7dc5009 in __libc_start_main () from /usr/lib/libc.so.6 28 0x000000000a48702e in _start () at new:340 --- base/daemon/BaseDaemon.cpp | 9 ++++++++- base/daemon/BaseDaemon.h | 2 ++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index fa81c9aeaf9..78801e71a6f 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -459,6 +459,11 @@ BaseDaemon::~BaseDaemon() { writeSignalIDtoSignalPipe(SignalListener::StopThread); signal_listener_thread.join(); + /// Reset signals to SIG_DFL to avoid trying to write to the signal_pipe that will be closed after. + for (int sig : handled_signals) + { + signal(sig, SIG_DFL); + } signal_pipe.close(); } @@ -713,7 +718,7 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() /// Setup signal handlers. auto add_signal_handler = - [](const std::vector<int> & signals, signal_function handler) + [this](const std::vector<int> & signals, signal_function handler) { struct sigaction sa; memset(&sa, 0, sizeof(sa)); @@ -737,6 +742,8 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() for (auto signal : signals) if (sigaction(signal, &sa, nullptr)) throw Poco::Exception("Cannot set signal handler."); + + std::copy(signals.begin(), signals.end(), std::back_inserter(handled_signals)); } }; diff --git a/base/daemon/BaseDaemon.h b/base/daemon/BaseDaemon.h index 08bbfa291c4..f4d3f3dfe98 100644 --- a/base/daemon/BaseDaemon.h +++ b/base/daemon/BaseDaemon.h @@ -192,6 +192,8 @@ protected: Poco::Util::AbstractConfiguration * last_configuration = nullptr; String build_id_info; + + std::vector<int> handled_signals; }; From 52cf85eaee07f059ecd90fd34b5a762dfb308b59 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu, 13 Aug 2020 23:13:13 +0300 Subject: [PATCH 312/374] Fix clang build --- src/Common/tests/average.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/tests/average.cpp b/src/Common/tests/average.cpp index 0351c0a91a1..7462d06340a 100644 --- a/src/Common/tests/average.cpp +++ b/src/Common/tests/average.cpp @@ -9,9 +9,8 @@ #include <Common/Arena.h> #include <Common/Stopwatch.h> -#if !__clang__ #pragma GCC diagnostic ignored "-Wframe-larger-than=" -#endif + /** This test program evaluates different solutions for a simple degenerate task: * Aggregate data by UInt8 key, calculate "avg" function on Float values. From ad20a384c1bd3119cdff5f9a992a7ff6637eb335 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri, 14 Aug 2020 02:54:52 +0300 Subject: [PATCH 313/374] Remove useless code --- CMakeLists.txt | 1 - cmake/find/termcap.cmake | 8 -------- 2 files changed, 9 deletions(-) delete mode 100644 cmake/find/termcap.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index c43b881d482..c0889383b1d 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -368,7 +368,6 @@ include (cmake/find/icu.cmake) include (cmake/find/zlib.cmake) include (cmake/find/zstd.cmake) include (cmake/find/ltdl.cmake) # for odbc -include (cmake/find/termcap.cmake) # openssl, zlib before poco include (cmake/find/sparsehash.cmake) include (cmake/find/re2.cmake) diff --git a/cmake/find/termcap.cmake b/cmake/find/termcap.cmake deleted file mode 100644 index 7564b7134e7..00000000000 --- a/cmake/find/termcap.cmake +++ /dev/null @@ -1,8 +0,0 @@ -find_library (TERMCAP_LIBRARY tinfo) -if (NOT TERMCAP_LIBRARY) - find_library (TERMCAP_LIBRARY ncurses) -endif() -if (NOT TERMCAP_LIBRARY) - find_library (TERMCAP_LIBRARY termcap) -endif() -message (STATUS "Using termcap: ${TERMCAP_LIBRARY}") From 3b05c315e4be46a1043e497f3781b27dbae55a1e Mon Sep 17 00:00:00 2001 From: alexey-milovidov <milovidov@yandex-team.ru> Date: Fri, 14 Aug 2020 04:10:10 +0300 Subject: [PATCH 314/374] Update Pipe.h --- src/Processors/Pipe.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 6795d3b58b4..28b64937aeb 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -13,8 +13,8 @@ using StoragePtr = std::shared_ptr<IStorage>; using OutputPortRawPtrs = std::vector<OutputPort *>; -/// Pipes is a set of processors which represents the part of pipeline. -/// Pipes contains a list of output ports, with specified port for totals and specified port for extremes. +/// Pipe is a set of processors which represents the part of pipeline. +/// Pipe contains a list of output ports, with specified port for totals and specified port for extremes. /// All output ports have same header. /// All other ports are connected, all connections are inside processors set. class Pipe From baa85f0a8857f1b26dc0026d74eedd8dda8a6f6b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri, 14 Aug 2020 04:35:02 +0300 Subject: [PATCH 315/374] Fix timeout in sql_fuzzy test --- src/Functions/GatherUtils/Algorithms.h | 20 ++++++++++++++++++++ src/Functions/GatherUtils/GatherUtils.h | 1 - tests/queries/0_stateless/00746_sql_fuzzy.sh | 2 +- 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/src/Functions/GatherUtils/Algorithms.h b/src/Functions/GatherUtils/Algorithms.h index fbfed640e22..0f696ffacd8 100644 --- a/src/Functions/GatherUtils/Algorithms.h +++ b/src/Functions/GatherUtils/Algorithms.h @@ -12,11 +12,15 @@ namespace DB::ErrorCodes { extern const int LOGICAL_ERROR; + extern const int TOO_LARGE_ARRAY_SIZE; } namespace DB::GatherUtils { +inline constexpr size_t MAX_ARRAY_SIZE = 1 << 30; + + /// Methods to copy Slice to Sink, overloaded for various combinations of types. template <typename T> @@ -673,6 +677,10 @@ void resizeDynamicSize(ArraySource && array_source, ValueSource && value_source, if (size >= 0) { auto length = static_cast<size_t>(size); + if (length > MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}", + length, MAX_ARRAY_SIZE); + if (array_size <= length) { writeSlice(array_source.getWhole(), sink); @@ -685,6 +693,10 @@ void resizeDynamicSize(ArraySource && array_source, ValueSource && value_source, else { auto length = static_cast<size_t>(-size); + if (length > MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}", + length, MAX_ARRAY_SIZE); + if (array_size <= length) { for (size_t i = array_size; i < length; ++i) @@ -714,6 +726,10 @@ void resizeConstantSize(ArraySource && array_source, ValueSource && value_source if (size >= 0) { auto length = static_cast<size_t>(size); + if (length > MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}", + length, MAX_ARRAY_SIZE); + if (array_size <= length) { writeSlice(array_source.getWhole(), sink); @@ -726,6 +742,10 @@ void resizeConstantSize(ArraySource && array_source, ValueSource && value_source else { auto length = static_cast<size_t>(-size); + if (length > MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}", + length, MAX_ARRAY_SIZE); + if (array_size <= length) { for (size_t i = array_size; i < length; ++i) diff --git a/src/Functions/GatherUtils/GatherUtils.h b/src/Functions/GatherUtils/GatherUtils.h index be6fae017c0..6699cc655e4 100644 --- a/src/Functions/GatherUtils/GatherUtils.h +++ b/src/Functions/GatherUtils/GatherUtils.h @@ -57,7 +57,6 @@ void sliceHas(IArraySource & first, IArraySource & second, ArraySearchType searc void push(IArraySource & array_source, IValueSource & value_source, IArraySink & sink, bool push_front); void resizeDynamicSize(IArraySource & array_source, IValueSource & value_source, IArraySink & sink, const IColumn & size_column); - void resizeConstantSize(IArraySource & array_source, IValueSource & value_source, IArraySink & sink, ssize_t size); } diff --git a/tests/queries/0_stateless/00746_sql_fuzzy.sh b/tests/queries/0_stateless/00746_sql_fuzzy.sh index 10b4265b57f..aed00c905d7 100755 --- a/tests/queries/0_stateless/00746_sql_fuzzy.sh +++ b/tests/queries/0_stateless/00746_sql_fuzzy.sh @@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -q "select name from system.table_functions format TSV;" > "$ # if you want long run use: env SQL_FUZZY_RUNS=100000 clickhouse-test sql_fuzzy for SQL_FUZZY_RUN in $(seq "${SQL_FUZZY_RUNS:=10}"); do - env SQL_FUZZY_RUN="$SQL_FUZZY_RUN" "$CURDIR"/00746_sql_fuzzy.pl | $CLICKHOUSE_CLIENT --format Null --max_execution_time 10 -n --ignore-error >/dev/null 2>&1 + env SQL_FUZZY_RUN="$SQL_FUZZY_RUN" "$CURDIR"/00746_sql_fuzzy.pl | timeout 60 $CLICKHOUSE_CLIENT --format Null --max_execution_time 10 -n --ignore-error >/dev/null 2>&1 if [[ $($CLICKHOUSE_CLIENT -q "SELECT 'Still alive'") != 'Still alive' ]]; then break fi From de4a070d6a73098781ebf2720c5861c4a49b51f4 Mon Sep 17 00:00:00 2001 From: Amos Bird <amosbird@gmail.com> Date: Fri, 14 Aug 2020 09:54:51 +0800 Subject: [PATCH 316/374] log error message --- src/Storages/MergeTree/MergeTreeData.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fbf5f1424ef..173a1353f7e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1259,7 +1259,8 @@ void MergeTreeData::dropAllData() } catch (const Poco::FileNotFoundException &) { - /// If the file is already deleted, do nothing. + /// If the file is already deleted, log the error message and do nothing. + tryLogCurrentException(__PRETTY_FUNCTION__); } } From fe6696390c71e2f70b460e7472ce019efbdd4c09 Mon Sep 17 00:00:00 2001 From: Amos Bird <amosbird@gmail.com> Date: Fri, 14 Aug 2020 10:49:40 +0800 Subject: [PATCH 317/374] Fix shared build --- programs/install/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/install/CMakeLists.txt b/programs/install/CMakeLists.txt index 79f9558ccbb..c3f4d96d631 100644 --- a/programs/install/CMakeLists.txt +++ b/programs/install/CMakeLists.txt @@ -3,6 +3,7 @@ set (CLICKHOUSE_INSTALL_SOURCES Install.cpp) set (CLICKHOUSE_INSTALL_LINK PRIVATE boost::program_options + clickhouse_common_config clickhouse_common_io dbms readpassphrase From 7a6e0deea5cf2d4077a3074a1a643e44d27de943 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri, 14 Aug 2020 08:28:26 +0300 Subject: [PATCH 318/374] Fix build after merge --- src/Storages/StorageMaterializeMySQL.cpp | 16 ++++++---------- src/Storages/StorageMaterializeMySQL.h | 2 +- 2 files changed, 7 insertions(+), 11 deletions(-) diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index 92793df22a0..d6a99e73667 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -28,7 +28,7 @@ StorageMaterializeMySQL::StorageMaterializeMySQL(const StoragePtr & nested_stora setInMemoryMetadata(in_memory_metadata); } -Pipes StorageMaterializeMySQL::read( +Pipe StorageMaterializeMySQL::read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, const SelectQueryInfo & query_info, @@ -77,22 +77,18 @@ Pipes StorageMaterializeMySQL::read( expressions->children.emplace_back(std::make_shared<ASTIdentifier>(column_name)); } - Pipes pipes = nested_storage->read(require_columns_name, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); + Pipe pipe = nested_storage->read(require_columns_name, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); - if (!expressions->children.empty() && !pipes.empty()) + if (!expressions->children.empty() && !pipe.empty()) { - Block pipe_header = pipes.front().getHeader(); + Block pipe_header = pipe.getHeader(); auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList()); ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true); - for (auto & pipe : pipes) - { - assertBlocksHaveEqualStructure(pipe_header, pipe.getHeader(), "StorageMaterializeMySQL"); - pipe.addSimpleTransform(std::make_shared<FilterTransform>(pipe.getHeader(), expression_actions, filter_column_name, false)); - } + pipe.addTransform(std::make_shared<FilterTransform>(pipe.getHeader(), expression_actions, filter_column_name, false)); } - return pipes; + return pipe; } NamesAndTypesList StorageMaterializeMySQL::getVirtuals() const diff --git a/src/Storages/StorageMaterializeMySQL.h b/src/Storages/StorageMaterializeMySQL.h index d3e132844ee..4278ce64bd7 100644 --- a/src/Storages/StorageMaterializeMySQL.h +++ b/src/Storages/StorageMaterializeMySQL.h @@ -21,7 +21,7 @@ public: StorageMaterializeMySQL(const StoragePtr & nested_storage_, const DatabaseMaterializeMySQL * database_); - Pipes read( + Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; From 7f547b948699c1dc5cff7e88e897005bdeeafd31 Mon Sep 17 00:00:00 2001 From: alexey-milovidov <milovidov@yandex-team.ru> Date: Fri, 14 Aug 2020 08:41:34 +0300 Subject: [PATCH 319/374] Update adopters.md --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 6351e34b3a3..20bf38300ef 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -65,6 +65,7 @@ toc_title: Adopters | <a href="https://rambler.ru" class="favicon">Rambler</a> | Internet services | Analytics | — | — | [Talk in Russian, April 2018](https://medium.com/@ramblertop/разработка-api-clickhouse-для-рамблер-топ-100-f4c7e56f3141) | | <a href="https://www.s7.ru" class="favicon">S7 Airlines</a> | Airlines | Metrics, Logging | — | — | [Talk in Russian, March 2019](https://www.youtube.com/watch?v=nwG68klRpPg&t=15s) | | <a href="https://www.scireum.de/" class="favicon">scireum GmbH</a> | e-Commerce | Main product | — | — | [Talk in German, February 2020](https://www.youtube.com/watch?v=7QWAn5RbyR4) | +| <a href="https://segment.com/" class="favicon">Segment</a> | Data processing | Main product | — | — | [Slides, 2019](https://slides.com/abraithwaite/segment-clickhouse) | | <a href="https://www.semrush.com/" class="favicon">SEMrush</a> | Marketing | Main product | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/5_semrush.pdf) | | <a href="https://sentry.io/" class="favicon">Sentry</a> | Software Development | Main product | — | — | [Blog Post in English, May 2019](https://blog.sentry.io/2019/05/16/introducing-snuba-sentrys-new-search-infrastructure) | | <a href="https://seo.do/" class="favicon">seo.do</a> | Analytics | Main product | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/CH%20Presentation-%20Metehan%20Çetinkaya.pdf) | From 09a72d0c64ef1efeff54dc47b9943ea61008fb62 Mon Sep 17 00:00:00 2001 From: alexey-milovidov <milovidov@yandex-team.ru> Date: Fri, 14 Aug 2020 08:43:57 +0300 Subject: [PATCH 320/374] Update adopters.md --- docs/en/introduction/adopters.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 20bf38300ef..308e29c52ec 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -65,7 +65,7 @@ toc_title: Adopters | <a href="https://rambler.ru" class="favicon">Rambler</a> | Internet services | Analytics | — | — | [Talk in Russian, April 2018](https://medium.com/@ramblertop/разработка-api-clickhouse-для-рамблер-топ-100-f4c7e56f3141) | | <a href="https://www.s7.ru" class="favicon">S7 Airlines</a> | Airlines | Metrics, Logging | — | — | [Talk in Russian, March 2019](https://www.youtube.com/watch?v=nwG68klRpPg&t=15s) | | <a href="https://www.scireum.de/" class="favicon">scireum GmbH</a> | e-Commerce | Main product | — | — | [Talk in German, February 2020](https://www.youtube.com/watch?v=7QWAn5RbyR4) | -| <a href="https://segment.com/" class="favicon">Segment</a> | Data processing | Main product | — | — | [Slides, 2019](https://slides.com/abraithwaite/segment-clickhouse) | +| <a href="https://segment.com/" class="favicon">Segment</a> | Data processing | Main product | 9 * i3en.3xlarge nodes 7.5TB NVME SSDs, 96GB Memory, 12 vCPUs | — | [Slides, 2019](https://slides.com/abraithwaite/segment-clickhouse) | | <a href="https://www.semrush.com/" class="favicon">SEMrush</a> | Marketing | Main product | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/5_semrush.pdf) | | <a href="https://sentry.io/" class="favicon">Sentry</a> | Software Development | Main product | — | — | [Blog Post in English, May 2019](https://blog.sentry.io/2019/05/16/introducing-snuba-sentrys-new-search-infrastructure) | | <a href="https://seo.do/" class="favicon">seo.do</a> | Analytics | Main product | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/CH%20Presentation-%20Metehan%20Çetinkaya.pdf) | From 4c1e5d9d560ee642b59c1e8cf3cb8bb59b6cb838 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 14 Aug 2020 09:25:08 +0300 Subject: [PATCH 321/374] Fix cassandra build on macos --- cmake/find/cassandra.cmake | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/cmake/find/cassandra.cmake b/cmake/find/cassandra.cmake index f41e0f645f4..6ff7697ea57 100644 --- a/cmake/find/cassandra.cmake +++ b/cmake/find/cassandra.cmake @@ -1,6 +1,10 @@ option(ENABLE_CASSANDRA "Enable Cassandra" ${ENABLE_LIBRARIES}) if (ENABLE_CASSANDRA) + if (APPLE) + SET(CMAKE_MACOSX_RPATH ON) + endif() + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libuv") message (ERROR "submodule contrib/libuv is missing. to fix try run: \n git submodule update --init --recursive") elseif (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cassandra") From 8c85ab3753f936754b7eeec0811072c7380d39cf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri, 14 Aug 2020 09:52:28 +0300 Subject: [PATCH 322/374] Fix error with batch aggregation and -Array combinator --- src/AggregateFunctions/IAggregateFunction.h | 5 ++- src/Interpreters/Aggregator.cpp | 43 ++++++++++++------- .../01441_array_combinator.reference | 10 +++++ .../0_stateless/01441_array_combinator.sql | 1 + 4 files changed, 42 insertions(+), 17 deletions(-) create mode 100644 tests/queries/0_stateless/01441_array_combinator.reference create mode 100644 tests/queries/0_stateless/01441_array_combinator.sql diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index cb5f6604a93..d9c619809fc 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -388,13 +388,14 @@ public: { for (size_t j = 0; j < UNROLL_COUNT; ++j) { - if (has_data[j * 256 + k]) + size_t idx = j * 256 + k; + if (has_data[idx]) { AggregateDataPtr & place = map[k]; if (unlikely(!place)) init(place); - func.merge(place + place_offset, reinterpret_cast<const char *>(&places[256 * j + k]), arena); + func.merge(place + place_offset, reinterpret_cast<const char *>(&places[idx]), nullptr); } } } diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 9a0ee7fed86..8bfa4801a58 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -449,7 +449,6 @@ void NO_INLINE Aggregator::executeImpl( typename Method::State state(key_columns, key_sizes, aggregation_state_cache); if (!no_more_keys) - //executeImplCase<false>(method, state, aggregates_pool, rows, aggregate_instructions, overflow_row); executeImplBatch(method, state, aggregates_pool, rows, aggregate_instructions); else executeImplCase<true>(method, state, aggregates_pool, rows, aggregate_instructions, overflow_row); @@ -534,22 +533,36 @@ void NO_INLINE Aggregator::executeImplBatch( /// Optimization for special case when aggregating by 8bit key. if constexpr (std::is_same_v<Method, typename decltype(AggregatedDataVariants::key8)::element_type>) { + /// We use another method if there are aggregate functions with -Array combinator. + bool has_arrays = false; for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) { - inst->batch_that->addBatchLookupTable8( - rows, - reinterpret_cast<AggregateDataPtr *>(method.data.data()), - inst->state_offset, - [&](AggregateDataPtr & aggregate_data) - { - aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); - createAggregateStates(aggregate_data); - }, - state.getKeyData(), - inst->batch_arguments, - aggregates_pool); + if (inst->offsets) + { + has_arrays = true; + break; + } + } + + if (!has_arrays) + { + for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) + { + inst->batch_that->addBatchLookupTable8( + rows, + reinterpret_cast<AggregateDataPtr *>(method.data.data()), + inst->state_offset, + [&](AggregateDataPtr & aggregate_data) + { + aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + createAggregateStates(aggregate_data); + }, + state.getKeyData(), + inst->batch_arguments, + aggregates_pool); + } + return; } - return; } /// Generic case. @@ -629,7 +642,7 @@ void NO_INLINE Aggregator::executeOnIntervalWithoutKeyImpl( void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns & aggregate_columns, Columns & materialized_columns, - AggregateFunctionInstructions & aggregate_functions_instructions, NestedColumnsHolder & nested_columns_holder) + AggregateFunctionInstructions & aggregate_functions_instructions, NestedColumnsHolder & nested_columns_holder) { for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_columns[i].resize(params.aggregates[i].arguments.size()); diff --git a/tests/queries/0_stateless/01441_array_combinator.reference b/tests/queries/0_stateless/01441_array_combinator.reference new file mode 100644 index 00000000000..7f1bc308d22 --- /dev/null +++ b/tests/queries/0_stateless/01441_array_combinator.reference @@ -0,0 +1,10 @@ +0 0 +1 0 +2 0 +3 0 +4 0 +5 0 +6 0 +7 0 +8 0 +9 0 diff --git a/tests/queries/0_stateless/01441_array_combinator.sql b/tests/queries/0_stateless/01441_array_combinator.sql new file mode 100644 index 00000000000..68fd050940d --- /dev/null +++ b/tests/queries/0_stateless/01441_array_combinator.sql @@ -0,0 +1 @@ +SELECT number % 100 AS k, sumArray(emptyArrayUInt8()) AS v FROM numbers(10) GROUP BY k; From 89e967333662c8ec4c46dfa1ba12bd10781538b9 Mon Sep 17 00:00:00 2001 From: Artem Zuikov <chertus@gmail.com> Date: Fri, 14 Aug 2020 12:38:18 +0300 Subject: [PATCH 323/374] Hotfix for pushdown with StorageMerge (#13679) --- src/Interpreters/JoinedTables.cpp | 4 ++- .../PredicateExpressionsOptimizer.cpp | 5 +++ src/Storages/StorageMerge.cpp | 35 +++++++++++++++++-- src/Storages/VirtualColumnUtils.h | 1 - ...torage_merge_with_join_push_down.reference | 2 ++ ...1436_storage_merge_with_join_push_down.sql | 30 ++++++++++++++++ 6 files changed, 72 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01436_storage_merge_with_join_push_down.reference create mode 100644 tests/queries/0_stateless/01436_storage_merge_with_join_push_down.sql diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index 127df9b5eac..d38a3fa68dc 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -28,6 +28,7 @@ namespace ErrorCodes { extern const int ALIAS_REQUIRED; extern const int AMBIGUOUS_COLUMN_NAME; + extern const int LOGICAL_ERROR; } namespace @@ -187,7 +188,8 @@ StoragePtr JoinedTables::getLeftTableStorage() bool JoinedTables::resolveTables() { tables_with_columns = getDatabaseAndTablesWithColumns(table_expressions, context); - assert(tables_with_columns.size() == table_expressions.size()); + if (tables_with_columns.size() != table_expressions.size()) + throw Exception("Unexpected tables count", ErrorCodes::LOGICAL_ERROR); const auto & settings = context.getSettingsRef(); if (settings.joined_subquery_requires_alias && tables_with_columns.size() > 1) diff --git a/src/Interpreters/PredicateExpressionsOptimizer.cpp b/src/Interpreters/PredicateExpressionsOptimizer.cpp index 3915a0f7f43..86bdec628cd 100644 --- a/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -15,6 +15,7 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; } PredicateExpressionsOptimizer::PredicateExpressionsOptimizer( @@ -111,6 +112,10 @@ bool PredicateExpressionsOptimizer::tryRewritePredicatesToTables(ASTs & tables_e { bool is_rewrite_tables = false; + if (tables_element.size() != tables_predicates.size()) + throw Exception("Unexpected elements count in predicate push down: `set enable_optimize_predicate_expression = 0` to disable", + ErrorCodes::LOGICAL_ERROR); + for (size_t index = tables_element.size(); index > 0; --index) { size_t table_pos = index - 1; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index a98d789a048..2d96a59392b 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -37,6 +37,27 @@ namespace ErrorCodes extern const int SAMPLING_NOT_SUPPORTED; } +namespace +{ + +/// Rewrite original query removing joined tables from it +void removeJoin(const ASTSelectQuery & select) +{ + const auto & tables = select.tables(); + if (!tables || tables->children.size() < 2) + return; + + const auto & joined_table = tables->children[1]->as<ASTTablesInSelectQueryElement &>(); + if (!joined_table.table_join) + return; + + /// The most simple temporary solution: leave only the first table in query. + /// TODO: we also need to remove joined columns and related functions (taking in account aliases if any). + tables->children.resize(1); +} + +} + StorageMerge::StorageMerge( const StorageID & table_id_, @@ -243,6 +264,9 @@ Pipe StorageMerge::createSources( SelectQueryInfo modified_query_info = query_info; modified_query_info.query = query_info.query->clone(); + /// Original query could contain JOIN but we need only the first joined table and its columns. + removeJoin(*modified_query_info.query->as<ASTSelectQuery>()); + VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", table_name); Pipe pipe; @@ -428,9 +452,14 @@ Block StorageMerge::getQueryHeader( } case QueryProcessingStage::WithMergeableState: case QueryProcessingStage::Complete: - return InterpreterSelectQuery( - query_info.query, context, std::make_shared<OneBlockInputStream>(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())), - SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + { + auto query = query_info.query->clone(); + removeJoin(*query->as<ASTSelectQuery>()); + + auto stream = std::make_shared<OneBlockInputStream>( + metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())); + return InterpreterSelectQuery(query, context, stream, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + } } throw Exception("Logical Error: unknown processed stage.", ErrorCodes::LOGICAL_ERROR); } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index e1c7e400249..89b69eb79e3 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -26,7 +26,6 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va /// Leave in the block only the rows that fit under the WHERE clause and the PREWHERE clause of the query. /// Only elements of the outer conjunction are considered, depending only on the columns present in the block. -/// Returns true if at least one row is discarded. void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context); /// Extract from the input stream a set of `name` column values diff --git a/tests/queries/0_stateless/01436_storage_merge_with_join_push_down.reference b/tests/queries/0_stateless/01436_storage_merge_with_join_push_down.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/01436_storage_merge_with_join_push_down.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/01436_storage_merge_with_join_push_down.sql b/tests/queries/0_stateless/01436_storage_merge_with_join_push_down.sql new file mode 100644 index 00000000000..a3c598c6d83 --- /dev/null +++ b/tests/queries/0_stateless/01436_storage_merge_with_join_push_down.sql @@ -0,0 +1,30 @@ +DROP TABLE IF EXISTS test1; +DROP TABLE IF EXISTS test1_distributed; +DROP TABLE IF EXISTS test_merge; + +SET enable_optimize_predicate_expression = 1; + +CREATE TABLE test1 (id Int64, name String) ENGINE MergeTree PARTITION BY (id) ORDER BY (id); +CREATE TABLE test1_distributed AS test1 ENGINE = Distributed(test_cluster_two_shards_localhost, default, test1); +CREATE TABLE test_merge AS test1 ENGINE = Merge('default', 'test1_distributed'); + +SELECT count() FROM test_merge +JOIN (SELECT 'anystring' AS name) AS n +USING name +WHERE id = 1; + +DROP TABLE test1; +DROP TABLE test_merge; + + +CREATE TABLE test1 (id Int64, name String) ENGINE MergeTree PARTITION BY (id) ORDER BY (id); +CREATE TABLE test_merge AS test1 ENGINE = Merge('default', 'test1'); + +SELECT count() FROM test_merge +JOIN (SELECT 'anystring' AS name) AS n +USING name +WHERE id = 1; + +DROP TABLE test1; +DROP TABLE test_merge; +DROP TABLE test1_distributed; From e581b4b15aa258eabf5b829737dc5bc6b045d15d Mon Sep 17 00:00:00 2001 From: ylchou <ylchou@qq.com> Date: Fri, 14 Aug 2020 18:14:54 +0800 Subject: [PATCH 324/374] passwd and group location error There is no problem when the user is the same as the group by default. When the user and group are different, there will be a bug here. --- debian/clickhouse-server.init | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/debian/clickhouse-server.init b/debian/clickhouse-server.init index 7504b01d695..3c5b6edc05d 100755 --- a/debian/clickhouse-server.init +++ b/debian/clickhouse-server.init @@ -122,11 +122,11 @@ initdb() CLICKHOUSE_DATADIR_FROM_CONFIG=$CLICKHOUSE_DATADIR fi - if ! getent group ${CLICKHOUSE_USER} >/dev/null; then + if ! getent passwd ${CLICKHOUSE_USER} >/dev/null; then echo "Can't chown to non-existing user ${CLICKHOUSE_USER}" return fi - if ! getent passwd ${CLICKHOUSE_GROUP} >/dev/null; then + if ! getent group ${CLICKHOUSE_GROUP} >/dev/null; then echo "Can't chown to non-existing group ${CLICKHOUSE_GROUP}" return fi From d8d50f20fd63ba17b0e840f70c411ffc2aa7de22 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov <avtokmakov@yandex-team.ru> Date: Fri, 14 Aug 2020 14:04:47 +0300 Subject: [PATCH 325/374] fix sync 1 --- src/Storages/System/IStorageSystemOneBlock.h | 4 ++++ src/Storages/System/StorageSystemAsynchronousMetrics.h | 6 ++++++ 2 files changed, 10 insertions(+) diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 4e6e140e1b2..81650d669dc 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -21,6 +21,10 @@ protected: virtual void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const = 0; public: +#if defined(ARCADIA_BUILD) + IStorageSystemOneBlock(const String & name_) : IStorageSystemOneBlock(StorageID{"system", name_}) {} +#endif + IStorageSystemOneBlock(const StorageID & table_id_) : IStorage(table_id_) { StorageInMemoryMetadata metadata_; diff --git a/src/Storages/System/StorageSystemAsynchronousMetrics.h b/src/Storages/System/StorageSystemAsynchronousMetrics.h index 8f79c09b098..a2a92d248d8 100644 --- a/src/Storages/System/StorageSystemAsynchronousMetrics.h +++ b/src/Storages/System/StorageSystemAsynchronousMetrics.h @@ -25,6 +25,12 @@ private: const AsynchronousMetrics & async_metrics; protected: +#if defined(ARCADIA_BUILD) + StorageSystemAsynchronousMetrics(const String & name_, const AsynchronousMetrics & async_metrics_) + : StorageSystemAsynchronousMetrics(StorageID{"system", name_}, async_metrics_) + { + } +#endif StorageSystemAsynchronousMetrics(const StorageID & table_id_, const AsynchronousMetrics & async_metrics_); void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; From 22938a7152f495d25727ea4919bafd376e433dec Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 14 Aug 2020 15:22:47 +0300 Subject: [PATCH 326/374] Add separate docker container for unbundled build --- docker/images.json | 5 +++ docker/packager/deb/Dockerfile | 50 +++++-------------------- docker/packager/packager | 7 +++- docker/packager/unbundled/Dockerfile | 56 ++++++++++++++++++++++++++++ docker/packager/unbundled/build.sh | 28 ++++++++++++++ 5 files changed, 103 insertions(+), 43 deletions(-) create mode 100644 docker/packager/unbundled/Dockerfile create mode 100755 docker/packager/unbundled/build.sh diff --git a/docker/images.json b/docker/images.json index 0c4307f8e15..3eeff6b7c00 100644 --- a/docker/images.json +++ b/docker/images.json @@ -2,6 +2,7 @@ "docker/packager/deb": { "name": "yandex/clickhouse-deb-builder", "dependent": [ + "docker/packager/unbundled", "docker/test/stateless", "docker/test/stateless_with_coverage", "docker/test/stateless_pytest", @@ -15,6 +16,10 @@ "docker/test/pvs" ] }, + "docker/packager/unbundled": { + "name": "yandex/clickhouse-unbundled-builder", + "dependent": [] + }, "docker/test/coverage": { "name": "yandex/clickhouse-coverage", "dependent": [] diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index c1260b5c7ff..ac8caf2d1d0 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -1,9 +1,9 @@ # docker build -t yandex/clickhouse-deb-builder . -FROM ubuntu:19.10 +FROM ubuntu:20.04 RUN apt-get --allow-unauthenticated update -y && apt-get install --yes wget gnupg RUN wget -O - https://apt.llvm.org/llvm-snapshot.gpg.key | apt-key add - -RUN echo "deb [trusted=yes] http://apt.llvm.org/eoan/ llvm-toolchain-eoan-10 main" >> /etc/apt/sources.list +RUN echo "deb [trusted=yes] http://apt.llvm.org/focal/ llvm-toolchain-focal-11 main" >> /etc/apt/sources.list # initial packages RUN apt-get --allow-unauthenticated update -y \ @@ -25,13 +25,17 @@ RUN curl -O https://clickhouse-builds.s3.yandex.net/utils/1/dpkg-deb RUN chmod +x dpkg-deb RUN cp dpkg-deb /usr/bin - -# Libraries from OS are only needed to test the "unbundled" build (that is not used in production). RUN apt-get --allow-unauthenticated update -y \ && env DEBIAN_FRONTEND=noninteractive \ apt-get --allow-unauthenticated install --yes --no-install-recommends \ + gcc-10 \ + g++-10 \ gcc-9 \ g++-9 \ + llvm-11 \ + clang-11 \ + lld-11 \ + clang-tidy-11 \ llvm-10 \ clang-10 \ lld-10 \ @@ -39,54 +43,18 @@ RUN apt-get --allow-unauthenticated update -y \ clang-9 \ lld-9 \ clang-tidy-9 \ - libicu-dev \ - libreadline-dev \ - gperf \ ninja-build \ perl \ pkg-config \ devscripts \ debhelper \ git \ - libc++-dev \ - libc++abi-dev \ - libboost-program-options-dev \ - libboost-system-dev \ - libboost-filesystem-dev \ - libboost-thread-dev \ - libboost-iostreams-dev \ - libboost-regex-dev \ - zlib1g-dev \ - liblz4-dev \ - libdouble-conversion-dev \ - librdkafka-dev \ - libpoconetssl62 \ - libpoco-dev \ - libgoogle-perftools-dev \ - libzstd-dev \ - libltdl-dev \ - libre2-dev \ - libjemalloc-dev \ - libmsgpack-dev \ - libcurl4-openssl-dev \ - opencl-headers \ - ocl-icd-libopencl1 \ - intel-opencl-icd \ - unixodbc-dev \ - odbcinst \ tzdata \ gperf \ alien \ - libcapnp-dev \ cmake \ gdb \ - pigz \ - moreutils \ - libcctz-dev \ - libldap2-dev \ - libsasl2-dev \ - heimdal-multidev \ - libhyperscan-dev + moreutils # This symlink required by gcc to find lld compiler diff --git a/docker/packager/packager b/docker/packager/packager index bc97429336c..63652da2439 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -11,6 +11,7 @@ SCRIPT_PATH = os.path.realpath(__file__) IMAGE_MAP = { "deb": "yandex/clickhouse-deb-builder", "binary": "yandex/clickhouse-binary-builder", + "unbundled": "yandex/clickhouse-unbundled-builder" } def check_image_exists_locally(image_name): @@ -176,7 +177,9 @@ if __name__ == "__main__": parser.add_argument("--clickhouse-repo-path", default=os.path.join(os.path.dirname(os.path.abspath(__file__)), os.pardir, os.pardir)) parser.add_argument("--output-dir", required=True) parser.add_argument("--build-type", choices=("debug", ""), default="") - parser.add_argument("--compiler", choices=("clang-10-darwin", "clang-10-aarch64", "clang-10-freebsd", "gcc-9", "clang-10"), default="gcc-9") + parser.add_argument("--compiler", choices=("clang-10", "clang-10-darwin", "clang-10-aarch64", "clang-10-freebsd", + "clang-11", "clang-11-darwin", "clang-11-aarch64", "clang-11-freebsd", + "gcc-9", "gcc-10"), default="gcc-9") parser.add_argument("--sanitizer", choices=("address", "thread", "memory", "undefined", ""), default="") parser.add_argument("--unbundled", action="store_true") parser.add_argument("--split-binary", action="store_true") @@ -197,7 +200,7 @@ if __name__ == "__main__": if not os.path.isabs(args.output_dir): args.output_dir = os.path.abspath(os.path.join(os.getcwd(), args.output_dir)) - image_type = 'binary' if args.package_type == 'performance' else args.package_type + image_type = 'binary' if args.package_type == 'performance' else 'unbundled' if args.unbundled else args.package_type image_name = IMAGE_MAP[image_type] if not os.path.isabs(args.clickhouse_repo_path): diff --git a/docker/packager/unbundled/Dockerfile b/docker/packager/unbundled/Dockerfile new file mode 100644 index 00000000000..d099be6cb90 --- /dev/null +++ b/docker/packager/unbundled/Dockerfile @@ -0,0 +1,56 @@ +# docker build -t yandex/clickhouse-unbundled-builder . +FROM yandex/clickhouse-deb-builder + +# Libraries from OS are only needed to test the "unbundled" build (that is not used in production). +RUN apt-get --allow-unauthenticated update -y \ + && env DEBIAN_FRONTEND=noninteractive \ + apt-get --allow-unauthenticated install --yes --no-install-recommends \ + libicu-dev \ + libreadline-dev \ + gperf \ + perl \ + pkg-config \ + devscripts \ + libc++-dev \ + libc++abi-dev \ + libboost-program-options-dev \ + libboost-system-dev \ + libboost-filesystem-dev \ + libboost-thread-dev \ + libboost-iostreams-dev \ + libboost-regex-dev \ + zlib1g-dev \ + liblz4-dev \ + libdouble-conversion-dev \ + librdkafka-dev \ + libpoconetssl62 \ + libpoco-dev \ + libgoogle-perftools-dev \ + libzstd-dev \ + libltdl-dev \ + libre2-dev \ + libjemalloc-dev \ + libmsgpack-dev \ + libcurl4-openssl-dev \ + opencl-headers \ + ocl-icd-libopencl1 \ + intel-opencl-icd \ + unixodbc-dev \ + odbcinst \ + tzdata \ + gperf \ + alien \ + libcapnp-dev \ + cmake \ + gdb \ + pigz \ + moreutils \ + libcctz-dev \ + libldap2-dev \ + libsasl2-dev \ + heimdal-multidev \ + libhyperscan-dev + +COPY build.sh / + +CMD ["/bin/bash", "/build.sh"] diff --git a/docker/packager/unbundled/build.sh b/docker/packager/unbundled/build.sh new file mode 100755 index 00000000000..8b26bbb19cb --- /dev/null +++ b/docker/packager/unbundled/build.sh @@ -0,0 +1,28 @@ +#!/usr/bin/env bash + +set -x -e + +# Update tzdata to the latest version. It is embedded into clickhouse binary. +sudo apt-get update && sudo apt-get install tzdata + +ccache --show-stats ||: +ccache --zero-stats ||: +build/release --no-pbuilder $ALIEN_PKGS | ts '%Y-%m-%d %H:%M:%S' +mv /*.deb /output +mv *.changes /output +mv *.buildinfo /output +mv /*.rpm /output ||: # if exists +mv /*.tgz /output ||: # if exists + +if [ -n "$BINARY_OUTPUT" ] && { [ "$BINARY_OUTPUT" = "programs" ] || [ "$BINARY_OUTPUT" = "tests" ] ;} +then + echo Place $BINARY_OUTPUT to output + mkdir /output/binary ||: # if exists + mv /build/obj-*/programs/clickhouse* /output/binary + if [ "$BINARY_OUTPUT" = "tests" ] + then + mv /build/obj-*/src/unit_tests_dbms /output/binary + fi +fi +ccache --show-stats ||: +ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||: From b74ded3dc03439d52bc91cdf84f729281eabbb63 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Fri, 14 Aug 2020 15:34:00 +0300 Subject: [PATCH 327/374] Update build.sh --- docker/packager/unbundled/build.sh | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/docker/packager/unbundled/build.sh b/docker/packager/unbundled/build.sh index 8b26bbb19cb..9fd246bcc33 100755 --- a/docker/packager/unbundled/build.sh +++ b/docker/packager/unbundled/build.sh @@ -14,15 +14,5 @@ mv *.buildinfo /output mv /*.rpm /output ||: # if exists mv /*.tgz /output ||: # if exists -if [ -n "$BINARY_OUTPUT" ] && { [ "$BINARY_OUTPUT" = "programs" ] || [ "$BINARY_OUTPUT" = "tests" ] ;} -then - echo Place $BINARY_OUTPUT to output - mkdir /output/binary ||: # if exists - mv /build/obj-*/programs/clickhouse* /output/binary - if [ "$BINARY_OUTPUT" = "tests" ] - then - mv /build/obj-*/src/unit_tests_dbms /output/binary - fi -fi ccache --show-stats ||: ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||: From ab337b243a0255f7a031341763a23258c2ebd274 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 14 Aug 2020 16:09:23 +0300 Subject: [PATCH 328/374] Add docker for style check --- docker/images.json | 4 ++++ docker/test/style/Dockerfile | 8 ++++++++ utils/check-style/check-duplicate-includes.sh | 6 ------ 3 files changed, 12 insertions(+), 6 deletions(-) create mode 100644 docker/test/style/Dockerfile diff --git a/docker/images.json b/docker/images.json index 0c4307f8e15..0ab1688efb0 100644 --- a/docker/images.json +++ b/docker/images.json @@ -92,6 +92,10 @@ "name": "yandex/clickhouse-fasttest", "dependent": [] }, + "docker/test/style": { + "name": "yandex/clickhouse-style-test", + "dependent": [] + }, "docker/test/integration/s3_proxy": { "name": "yandex/clickhouse-s3-proxy", "dependent": [] diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile new file mode 100644 index 00000000000..d80a1e89aac --- /dev/null +++ b/docker/test/style/Dockerfile @@ -0,0 +1,8 @@ +# docker build -t yandex/clickhouse-style-test . +FROM ubuntu:20.04 + +RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes shellcheck libxml2-utils git + + +CMD cd /ClickHouse/utils/check-style && ./check-style -n | tee /test_output/style_output.txt && \ + ./check-duplicate-includes.sh | tee /test_output/duplicate_output.txt diff --git a/utils/check-style/check-duplicate-includes.sh b/utils/check-style/check-duplicate-includes.sh index df843ead623..e69de29bb2d 100755 --- a/utils/check-style/check-duplicate-includes.sh +++ b/utils/check-style/check-duplicate-includes.sh @@ -1,6 +0,0 @@ -#!/usr/bin/env bash - -ROOT_PATH=$(git rev-parse --show-toplevel) - -# Find duplicate include directives -find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | while read file; do grep -P '^#include ' $file | sort | uniq -c | grep -v -P '^\s+1\s' && echo $file; done From 76df13c31c2a1165694e969d21bac4e2cdc3cbfe Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov <nik-kochetov@yandex-team.ru> Date: Fri, 14 Aug 2020 16:16:36 +0300 Subject: [PATCH 329/374] Fix tests. --- src/Storages/StorageMaterializeMySQL.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index d6a99e73667..4eeb4975185 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -85,7 +85,10 @@ Pipe StorageMaterializeMySQL::read( auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList()); ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true); - pipe.addTransform(std::make_shared<FilterTransform>(pipe.getHeader(), expression_actions, filter_column_name, false)); + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared<FilterTransform>(header, expression_actions, filter_column_name, false); + }); } return pipe; From 03eb8f5f21d91151f856105f8841cc75c6e19df9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov <avtokmakov@yandex-team.ru> Date: Fri, 14 Aug 2020 16:51:32 +0300 Subject: [PATCH 330/374] fix sync 2 --- .../System/StorageSystemDetachedParts.cpp | 2 +- src/Storages/System/attachSystemTables.cpp | 27 +------------------ src/Storages/System/attachSystemTablesImpl.h | 27 +++++++++++++++++++ 3 files changed, 29 insertions(+), 27 deletions(-) create mode 100644 src/Storages/System/attachSystemTablesImpl.h diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 2b74d5a1caf..5a233ed7a33 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -30,7 +30,7 @@ StorageSystemDetachedParts::StorageSystemDetachedParts(const StorageID & table_i setInMemoryMetadata(storage_metadata); } -Pipe read( +Pipe StorageSystemDetachedParts::read( const Names & /* column_names */, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 15edd096b2f..758feca7713 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -1,5 +1,6 @@ #include <Databases/IDatabase.h> #include <Storages/System/attachSystemTables.h> +#include <Storages/System/attachSystemTablesImpl.h> #include <Storages/System/StorageSystemAggregateFunctionCombinators.h> #include <Storages/System/StorageSystemAsynchronousMetrics.h> @@ -66,32 +67,6 @@ namespace DB { -namespace -{ - -template<typename StorageT, typename... StorageArgs> -void attach(IDatabase & system_database, const String & table_name, StorageArgs && ... args) -{ - if (system_database.getUUID() == UUIDHelpers::Nil) - { - /// Attach to Ordinary database - auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name); - system_database.attachTable(table_name, StorageT::create(table_id, std::forward<StorageArgs>(args)...)); - } - else - { - /// Attach to Atomic database - /// NOTE: UUIDs are not persistent, but it's ok since no data are stored on disk for these storages - /// and path is actually not used - auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name, UUIDHelpers::generateV4()); - String path = "store/" + DatabaseCatalog::getPathForUUID(table_id.uuid); - system_database.attachTable(table_name, StorageT::create(table_id, std::forward<StorageArgs>(args)...), path); - } -} - -} - - void attachSystemTablesLocal(IDatabase & system_database) { attach<StorageSystemOne>(system_database, "one"); diff --git a/src/Storages/System/attachSystemTablesImpl.h b/src/Storages/System/attachSystemTablesImpl.h new file mode 100644 index 00000000000..0dc34c3c116 --- /dev/null +++ b/src/Storages/System/attachSystemTablesImpl.h @@ -0,0 +1,27 @@ +#pragma once +#include <Databases/IDatabase.h> + +namespace DB +{ + +template<typename StorageT, typename... StorageArgs> +void attach(IDatabase & system_database, const String & table_name, StorageArgs && ... args) +{ + if (system_database.getUUID() == UUIDHelpers::Nil) + { + /// Attach to Ordinary database + auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name); + system_database.attachTable(table_name, StorageT::create(table_id, std::forward<StorageArgs>(args)...)); + } + else + { + /// Attach to Atomic database + /// NOTE: UUIDs are not persistent, but it's ok since no data are stored on disk for these storages + /// and path is actually not used + auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name, UUIDHelpers::generateV4()); + String path = "store/" + DatabaseCatalog::getPathForUUID(table_id.uuid); + system_database.attachTable(table_name, StorageT::create(table_id, std::forward<StorageArgs>(args)...), path); + } +} + +} From ede56a75036477d4d24b25d43f2976a541a7e4fc Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 14 Aug 2020 17:41:03 +0300 Subject: [PATCH 331/374] Fix --- utils/check-style/check-duplicate-includes.sh | 6 ++++++ utils/check-style/check-style | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/utils/check-style/check-duplicate-includes.sh b/utils/check-style/check-duplicate-includes.sh index e69de29bb2d..df843ead623 100755 --- a/utils/check-style/check-duplicate-includes.sh +++ b/utils/check-style/check-duplicate-includes.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +ROOT_PATH=$(git rev-parse --show-toplevel) + +# Find duplicate include directives +find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | while read file; do grep -P '^#include ' $file | sort | uniq -c | grep -v -P '^\s+1\s' && echo $file; done diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 939eb3ecca8..0579dcf0c26 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -68,3 +68,9 @@ find $ROOT_PATH -name '.gitmodules' | while read i; do grep -F 'url = ' $i | gre # There shouldn't be any code snippets under GPL or LGPL find $ROOT_PATH/{src,base,programs} -name '*.h' -or -name '*.cpp' 2>/dev/null | xargs grep -i -F 'General Public License' && echo "There shouldn't be any code snippets under GPL or LGPL" + +# There shouldn't be any docker containers outside docker directory +find $ROOT_PATH -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' -name Dockerfile -type f 2>/dev/null | xargs --no-run-if-empty -n1 echo "Please move Dockerfile to docker directory:" + +# There shouldn't be any docker compose files outside docker directory +find $ROOT_PATH -not -path $ROOT_PATH'/tests/testflows*' -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' -name '*compose*.yml' -type f 2>/dev/null | xargs --no-run-if-empty grep -l "version:" | xargs --no-run-if-empty -n1 echo "Please move docker compose to docker directory:" From 194d1d514cdbef66594a39fad455754bb8ace343 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin <a3at.mail@gmail.com> Date: Fri, 14 Aug 2020 22:09:37 +0300 Subject: [PATCH 332/374] Fix compilation MySQLReplication with gcc10 and libstdc++ MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit gcc10 reports: ../src/Core/MySQLReplication.cpp:62:46: error: use of deleted function ‘std::basic_ostream<char, _Traits>& std::operator<<(std::basic_ostream<char, _Traits>&, char8_t) [with _Traits = std::char_traits<char>]’ --- src/Core/MySQLReplication.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Core/MySQLReplication.cpp b/src/Core/MySQLReplication.cpp index db0c70d07a1..e0721010c85 100644 --- a/src/Core/MySQLReplication.cpp +++ b/src/Core/MySQLReplication.cpp @@ -59,7 +59,7 @@ namespace MySQLReplication out << "Binlog Version: " << this->binlog_version << std::endl; out << "Server Version: " << this->server_version << std::endl; out << "Create Timestamp: " << this->create_timestamp << std::endl; - out << "Event Header Len: " << this->event_header_length << std::endl; + out << "Event Header Len: " << std::to_string(this->event_header_length) << std::endl; } /// https://dev.mysql.com/doc/internals/en/rotate-event.html @@ -119,7 +119,7 @@ namespace MySQLReplication header.dump(out); out << "Thread ID: " << this->thread_id << std::endl; out << "Execution Time: " << this->exec_time << std::endl; - out << "Schema Len: " << this->schema_len << std::endl; + out << "Schema Len: " << std::to_string(this->schema_len) << std::endl; out << "Error Code: " << this->error_code << std::endl; out << "Status Len: " << this->status_len << std::endl; out << "Schema: " << this->schema << std::endl; @@ -239,14 +239,14 @@ namespace MySQLReplication header.dump(out); out << "Table ID: " << this->table_id << std::endl; out << "Flags: " << this->flags << std::endl; - out << "Schema Len: " << this->schema_len << std::endl; + out << "Schema Len: " << std::to_string(this->schema_len) << std::endl; out << "Schema: " << this->schema << std::endl; - out << "Table Len: " << this->table_len << std::endl; + out << "Table Len: " << std::to_string(this->table_len) << std::endl; out << "Table: " << this->table << std::endl; out << "Column Count: " << this->column_count << std::endl; for (auto i = 0U; i < column_count; i++) { - out << "Column Type [" << i << "]: " << column_type[i] << ", Meta: " << column_meta[i] << std::endl; + out << "Column Type [" << i << "]: " << std::to_string(column_type[i]) << ", Meta: " << column_meta[i] << std::endl; } out << "Null Bitmap: " << this->null_bitmap << std::endl; } From c97a9de7e298b8eaaeeef5db176d672353742bb7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin <a3at.mail@gmail.com> Date: Fri, 14 Aug 2020 22:14:42 +0300 Subject: [PATCH 333/374] Fix -Wpessimizing-move in InterpretersMySQLDDLQuery gcc10 reports: ../src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp:354:44: warning: moving a local object in a return statement prevents copy elision [-Wpessimizing-move] 354 | return std::move(column_declaration); --- src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 9320e8eacf5..9e94e8f579e 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -351,7 +351,7 @@ ASTs InterpreterCreateImpl::getRewrittenQueries( column_declaration->default_expression = std::make_shared<ASTLiteral>(default_value); column_declaration->children.emplace_back(column_declaration->type); column_declaration->children.emplace_back(column_declaration->default_expression); - return std::move(column_declaration); + return column_declaration; }; /// Add _sign and _version column. From 52d31b9cc837052890febfded4709f5cf2abdb20 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev <deniszhuravlov@gmail.com> Date: Fri, 14 Aug 2020 16:54:13 -0300 Subject: [PATCH 334/374] Update settings.md removed remark about default's profile for max_server_memory_usage --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 0c4389fe0c1..63319ed2c01 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -374,7 +374,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat ## max_server_memory_usage {#max_server_memory_usage} -Ограничивает объём оперативной памяти, используемой сервером ClickHouse. Настройка может быть задана только для профиля `default`. +Ограничивает объём оперативной памяти, используемой сервером ClickHouse. Возможные значения: From 53422fe239c73880af1f3ab1d21df0af0a0c3570 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev <deniszhuravlov@gmail.com> Date: Fri, 14 Aug 2020 16:57:44 -0300 Subject: [PATCH 335/374] Update settings.md removed remark about default's profile for max_server_memory_usage --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index c49ead23c09..80d4d659bd3 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -399,7 +399,7 @@ The cache is shared for the server and memory is allocated as needed. The cache ``` ## max\_server\_memory\_usage {#max_server_memory_usage} -Limits total RAM usage by the ClickHouse server. You can specify it only for the default profile. +Limits total RAM usage by the ClickHouse server. Possible values: From 6fd0faa00fc5bfbbf98149e1c1ad40951c99b402 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin <a3at.mail@gmail.com> Date: Fri, 14 Aug 2020 23:41:22 +0300 Subject: [PATCH 336/374] Fix performance-no-automatic-move in InterpretersMySQLDDLQuery --- src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 9e94e8f579e..461dd997cd1 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -344,7 +344,7 @@ ASTs InterpreterCreateImpl::getRewrittenQueries( const auto & create_materialized_column_declaration = [&](const String & name, const String & type, const auto & default_value) { - const auto column_declaration = std::make_shared<ASTColumnDeclaration>(); + auto column_declaration = std::make_shared<ASTColumnDeclaration>(); column_declaration->name = name; column_declaration->type = makeASTFunction(type); column_declaration->default_specifier = "MATERIALIZED"; From df359bd289b64363619868f50d235872c4f3ded1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin <a3at.mail@gmail.com> Date: Sat, 15 Aug 2020 00:27:05 +0300 Subject: [PATCH 337/374] Fix logging Settings.Names/Values when log_queries_min_type > QUERY_START --- src/Interpreters/executeQuery.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 78e9f4bba3f..0f31e698421 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -474,13 +474,16 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl( bool log_queries = settings.log_queries && !internal; /// Log into system table start of query execution, if need. - if (log_queries && elem.type >= settings.log_queries_min_type) + if (log_queries) { if (settings.log_query_settings) elem.query_settings = std::make_shared<Settings>(context.getSettingsRef()); - if (auto query_log = context.getQueryLog()) - query_log->add(elem); + if (elem.type >= settings.log_queries_min_type) + { + if (auto query_log = context.getQueryLog()) + query_log->add(elem); + } } /// Common code for finish and exception callbacks From 249ceec61b933dff878f4aaa0dbe91fc2f0bed96 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin <a3at.mail@gmail.com> Date: Sat, 15 Aug 2020 00:30:58 +0300 Subject: [PATCH 338/374] Fix 01231_log_queries_min_type test --- .../0_stateless/01231_log_queries_min_type.reference | 4 ++-- tests/queries/0_stateless/01231_log_queries_min_type.sql | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01231_log_queries_min_type.reference b/tests/queries/0_stateless/01231_log_queries_min_type.reference index a358d022033..dc820b2eb1e 100644 --- a/tests/queries/0_stateless/01231_log_queries_min_type.reference +++ b/tests/queries/0_stateless/01231_log_queries_min_type.reference @@ -1,5 +1,5 @@ 01231_log_queries_min_type/QUERY_START 2 01231_log_queries_min_type/EXCEPTION_BEFORE_START -2 -3 +0 +1 diff --git a/tests/queries/0_stateless/01231_log_queries_min_type.sql b/tests/queries/0_stateless/01231_log_queries_min_type.sql index f2229c94a8a..0582499d242 100644 --- a/tests/queries/0_stateless/01231_log_queries_min_type.sql +++ b/tests/queries/0_stateless/01231_log_queries_min_type.sql @@ -2,14 +2,14 @@ set log_queries=1; select '01231_log_queries_min_type/QUERY_START'; system flush logs; -select count() from system.query_log where query like '%01231_log_queries_min_type/%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute; +select count() from system.query_log where query like '%01231_log_queries_min_type/QUERY_START%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute; set log_queries_min_type='EXCEPTION_BEFORE_START'; select '01231_log_queries_min_type/EXCEPTION_BEFORE_START'; system flush logs; -select count() from system.query_log where query like '%01231_log_queries_min_type/%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute; +select count() from system.query_log where query like '%01231_log_queries_min_type/EXCEPTION_BEFORE_START%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute; set log_queries_min_type='EXCEPTION_WHILE_PROCESSING'; -select '01231_log_queries_min_type/', max(number) from system.numbers limit 1e6 settings max_rows_to_read='100K'; -- { serverError 158; } +select '01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6 settings max_rows_to_read='100K'; -- { serverError 158; } system flush logs; -select count() from system.query_log where query like '%01231_log_queries_min_type/%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute; +select count() from system.query_log where query like '%01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute and type = 'ExceptionWhileProcessing'; From 082dbea0393774ef96c044db762133a8d895c814 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin <a3at.mail@gmail.com> Date: Sat, 15 Aug 2020 00:34:12 +0300 Subject: [PATCH 339/374] Cover Settings in query_log with non default log_queries_min_type --- .../01231_log_queries_min_type.reference | 1 + .../0_stateless/01231_log_queries_min_type.sql | 13 ++++++++++++- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01231_log_queries_min_type.reference b/tests/queries/0_stateless/01231_log_queries_min_type.reference index dc820b2eb1e..61f4c09dd20 100644 --- a/tests/queries/0_stateless/01231_log_queries_min_type.reference +++ b/tests/queries/0_stateless/01231_log_queries_min_type.reference @@ -3,3 +3,4 @@ 01231_log_queries_min_type/EXCEPTION_BEFORE_START 0 1 +1 diff --git a/tests/queries/0_stateless/01231_log_queries_min_type.sql b/tests/queries/0_stateless/01231_log_queries_min_type.sql index 0582499d242..bfeeca96d4b 100644 --- a/tests/queries/0_stateless/01231_log_queries_min_type.sql +++ b/tests/queries/0_stateless/01231_log_queries_min_type.sql @@ -9,7 +9,18 @@ select '01231_log_queries_min_type/EXCEPTION_BEFORE_START'; system flush logs; select count() from system.query_log where query like '%01231_log_queries_min_type/EXCEPTION_BEFORE_START%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute; +set max_rows_to_read='100K'; set log_queries_min_type='EXCEPTION_WHILE_PROCESSING'; -select '01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6 settings max_rows_to_read='100K'; -- { serverError 158; } +select '01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } system flush logs; select count() from system.query_log where query like '%01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute and type = 'ExceptionWhileProcessing'; + +select '01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } +system flush logs; +select count() from system.query_log where + query like '%01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING%' and + query not like '%system.query_log%' and + event_date = today() and + event_time >= now() - interval 1 minute and + type = 'ExceptionWhileProcessing' and + has(Settings.Names, 'max_rows_to_read'); From 1cbfd1327ac7ad7c4629e6b108661ef2ead4e5a4 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Sat, 15 Aug 2020 01:40:40 +0300 Subject: [PATCH 340/374] trigger the CI --- docker/test/style/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index d80a1e89aac..0f5c8d35fa9 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -1,4 +1,4 @@ -# docker build -t yandex/clickhouse-style-test . +# docker build -t yandex/clickhouse-style-test . FROM ubuntu:20.04 RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes shellcheck libxml2-utils git From 62ed3a16d03febd2347e2652b93230fadfa5688c Mon Sep 17 00:00:00 2001 From: alexey-milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 03:08:28 +0300 Subject: [PATCH 341/374] Update developer-instruction.md --- docs/ru/development/developer-instruction.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 025772ea3ca..6775b5378cb 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -1,3 +1,5 @@ +# Инструкция для разработчиков + Сборка ClickHouse поддерживается на Linux, FreeBSD, Mac OS X. # Если вы используете Windows {#esli-vy-ispolzuete-windows} From 52c3857225380123a7133b99ee07499462a8fcd3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 06:10:57 +0300 Subject: [PATCH 342/374] Fix server restart after stress test --- src/Databases/DatabaseWithDictionaries.cpp | 1 - .../CassandraDictionarySource.cpp | 1 + .../ClickHouseDictionarySource.cpp | 11 +++++++--- src/Dictionaries/ClickHouseDictionarySource.h | 3 ++- src/Dictionaries/DictionaryFactory.cpp | 3 ++- src/Dictionaries/DictionarySourceFactory.cpp | 3 ++- src/Dictionaries/DictionarySourceFactory.h | 5 +++++ .../ExecutableDictionarySource.cpp | 1 + src/Dictionaries/FileDictionarySource.cpp | 1 + src/Dictionaries/HTTPDictionarySource.cpp | 1 + src/Dictionaries/LibraryDictionarySource.cpp | 1 + src/Dictionaries/MongoDBDictionarySource.cpp | 1 + src/Dictionaries/MySQLDictionarySource.cpp | 1 + src/Dictionaries/RedisDictionarySource.cpp | 1 + src/Dictionaries/XDBCDictionarySource.cpp | 2 ++ .../getDictionaryConfigurationFromAST.cpp | 10 ++++++++-- ..._expressions_in_engine_arguments.reference | 20 +++++++++---------- .../01083_expressions_in_engine_arguments.sql | 8 +------- 18 files changed, 48 insertions(+), 26 deletions(-) diff --git a/src/Databases/DatabaseWithDictionaries.cpp b/src/Databases/DatabaseWithDictionaries.cpp index f724faac637..ed85028d04d 100644 --- a/src/Databases/DatabaseWithDictionaries.cpp +++ b/src/Databases/DatabaseWithDictionaries.cpp @@ -153,7 +153,6 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S if (isTableExist(dictionary_name, global_context)) throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", dict_id.getFullTableName()); - String dictionary_metadata_path = getObjectMetadataPath(dictionary_name); String dictionary_metadata_tmp_path = dictionary_metadata_path + ".tmp"; String statement = getObjectDefinitionFromCreateQuery(query); diff --git a/src/Dictionaries/CassandraDictionarySource.cpp b/src/Dictionaries/CassandraDictionarySource.cpp index 5c7fd4f50fd..643e9af54b8 100644 --- a/src/Dictionaries/CassandraDictionarySource.cpp +++ b/src/Dictionaries/CassandraDictionarySource.cpp @@ -18,6 +18,7 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory) [[maybe_unused]] const std::string & config_prefix, [[maybe_unused]] Block & sample_block, const Context & /* context */, + const std::string & /* default_database */, bool /*check_config*/) -> DictionarySourcePtr { #if USE_CASSANDRA diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 180750d143a..d9a81631fca 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -53,7 +53,8 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( const std::string & path_to_settings, const std::string & config_prefix, const Block & sample_block_, - const Context & context_) + const Context & context_, + const std::string & default_database) : update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , host{config.getString(config_prefix + ".host")} @@ -61,7 +62,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( , secure(config.getBool(config_prefix + ".secure", false)) , user{config.getString(config_prefix + ".user", "")} , password{config.getString(config_prefix + ".password", "")} - , db{config.getString(config_prefix + ".db", "")} + , db{config.getString(config_prefix + ".db", default_database)} , table{config.getString(config_prefix + ".table")} , where{config.getString(config_prefix + ".where", "")} , update_field{config.getString(config_prefix + ".update_field", "")} @@ -73,6 +74,8 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( , pool{is_local ? nullptr : createPool(host, port, secure, db, user, password)} , load_all_query{query_builder.composeLoadAllQuery()} { + std::cerr << "DB: " << db << "\n"; + /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). context.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0)); context = copyContextAndApplySettings(path_to_settings, context, config); @@ -226,9 +229,11 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) const std::string & config_prefix, Block & sample_block, const Context & context, + const std::string & default_database, bool /* check_config */) -> DictionarySourcePtr { - return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix, config_prefix + ".clickhouse", sample_block, context); + return std::make_unique<ClickHouseDictionarySource>( + dict_struct, config, config_prefix, config_prefix + ".clickhouse", sample_block, context, default_database); }; factory.registerSource("clickhouse", create_table_source); } diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index 13dc0323039..276f6916ada 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -24,7 +24,8 @@ public: const std::string & path_to_settings, const std::string & config_prefix, const Block & sample_block_, - const Context & context); + const Context & context, + const std::string & default_database); /// copy-constructor is provided in order to support cloneability ClickHouseDictionarySource(const ClickHouseDictionarySource & other); diff --git a/src/Dictionaries/DictionaryFactory.cpp b/src/Dictionaries/DictionaryFactory.cpp index 03b4fe106c7..c33b7b5a3ae 100644 --- a/src/Dictionaries/DictionaryFactory.cpp +++ b/src/Dictionaries/DictionaryFactory.cpp @@ -42,7 +42,8 @@ DictionaryPtr DictionaryFactory::create( const DictionaryStructure dict_struct{config, config_prefix + ".structure"}; - DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create(name, config, config_prefix + ".source", dict_struct, context, check_source_config); + DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create( + name, config, config_prefix + ".source", dict_struct, context, config.getString(config_prefix + ".database", ""), check_source_config); LOG_TRACE(&Poco::Logger::get("DictionaryFactory"), "Created dictionary source '{}' for dictionary '{}'", source_ptr->toString(), name); const auto & layout_type = keys.front(); diff --git a/src/Dictionaries/DictionarySourceFactory.cpp b/src/Dictionaries/DictionarySourceFactory.cpp index a9e90024745..af3552364ba 100644 --- a/src/Dictionaries/DictionarySourceFactory.cpp +++ b/src/Dictionaries/DictionarySourceFactory.cpp @@ -80,6 +80,7 @@ DictionarySourcePtr DictionarySourceFactory::create( const std::string & config_prefix, const DictionaryStructure & dict_struct, const Context & context, + const std::string & default_database, bool check_config) const { Poco::Util::AbstractConfiguration::Keys keys; @@ -96,7 +97,7 @@ DictionarySourcePtr DictionarySourceFactory::create( { const auto & create_source = found->second; auto sample_block = createSampleBlock(dict_struct); - return create_source(dict_struct, config, config_prefix, sample_block, context, check_config); + return create_source(dict_struct, config, config_prefix, sample_block, context, default_database, check_config); } throw Exception{name + ": unknown dictionary source type: " + source_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG}; diff --git a/src/Dictionaries/DictionarySourceFactory.h b/src/Dictionaries/DictionarySourceFactory.h index 5d3578a7e28..0a68b7d6e05 100644 --- a/src/Dictionaries/DictionarySourceFactory.h +++ b/src/Dictionaries/DictionarySourceFactory.h @@ -26,12 +26,16 @@ class DictionarySourceFactory : private boost::noncopyable public: static DictionarySourceFactory & instance(); + /// 'default_database' - the database when dictionary itself was created. + /// It is used as default_database for ClickHouse dictionary source when no explicit database was specified. + /// Does not make sense for other sources. using Creator = std::function<DictionarySourcePtr( const DictionaryStructure & dict_struct, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, const Context & context, + const std::string & default_database, bool check_config)>; DictionarySourceFactory(); @@ -44,6 +48,7 @@ public: const std::string & config_prefix, const DictionaryStructure & dict_struct, const Context & context, + const std::string & default_database, bool check_config) const; private: diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 38965e00a84..46df227dd67 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -220,6 +220,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) const std::string & config_prefix, Block & sample_block, const Context & context, + const std::string & /* default_database */, bool check_config) -> DictionarySourcePtr { if (dict_struct.has_expressions) diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index a581f77ed3a..18893a99f4e 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -76,6 +76,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) const std::string & config_prefix, Block & sample_block, const Context & context, + const std::string & /* default_database */, bool check_config) -> DictionarySourcePtr { if (dict_struct.has_expressions) diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 95aab78ba2b..18a97f34486 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -197,6 +197,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) const std::string & config_prefix, Block & sample_block, const Context & context, + const std::string & /* default_database */, bool check_config) -> DictionarySourcePtr { if (dict_struct.has_expressions) diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index ba538201910..6d763444b54 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -298,6 +298,7 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory) const std::string & config_prefix, Block & sample_block, const Context & context, + const std::string & /* default_database */, bool check_config) -> DictionarySourcePtr { return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context, check_config); diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index f4376da952d..5b5d0c4d20f 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -14,6 +14,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) const std::string & root_config_prefix, Block & sample_block, const Context &, + const std::string & /* default_database */, bool /* check_config */) { const auto config_prefix = root_config_prefix + ".mongodb"; diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index f016f2bf537..a21b1bd50fc 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -19,6 +19,7 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) const std::string & config_prefix, Block & sample_block, const Context & /* context */, + const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { #if USE_MYSQL return std::make_unique<MySQLDictionarySource>(dict_struct, config, config_prefix + ".mysql", sample_block); diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 8794f0620e2..27acd723938 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -13,6 +13,7 @@ void registerDictionarySourceRedis(DictionarySourceFactory & factory) const String & config_prefix, Block & sample_block, const Context & /* context */, + const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { return std::make_unique<RedisDictionarySource>(dict_struct, config, config_prefix + ".redis", sample_block); }; diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index b3393d55e5d..793d8da7390 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -275,6 +275,7 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory) const std::string & config_prefix, Block & sample_block, const Context & context, + const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { #if USE_ODBC BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>( @@ -300,6 +301,7 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory) const std::string & /* config_prefix */, Block & /* sample_block */, const Context & /* context */, + const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { throw Exception{"Dictionary source of type `jdbc` is disabled until consistent support for nullable fields.", ErrorCodes::SUPPORT_IS_DISABLED}; diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index b3db806ca5e..b1962e48eea 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -402,7 +402,11 @@ void buildConfigurationFromFunctionWithKeyValueArguments( * </mysql> * </source> */ -void buildSourceConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const ASTFunctionWithKeyValueArguments * source, const ASTDictionarySettings * settings) +void buildSourceConfiguration( + AutoPtr<Document> doc, + AutoPtr<Element> root, + const ASTFunctionWithKeyValueArguments * source, + const ASTDictionarySettings * settings) { AutoPtr<Element> outer_element(doc->createElement("source")); root->appendChild(outer_element); @@ -498,7 +502,9 @@ DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuer bool complex = DictionaryFactory::instance().isComplex(dictionary_layout->layout_type); - auto all_attr_names_and_types = buildDictionaryAttributesConfiguration(xml_document, structure_element, query.dictionary_attributes_list, pk_attrs); + auto all_attr_names_and_types = buildDictionaryAttributesConfiguration( + xml_document, structure_element, query.dictionary_attributes_list, pk_attrs); + checkPrimaryKey(all_attr_names_and_types, pk_attrs); buildPrimaryKeyConfiguration(xml_document, structure_element, complex, pk_attrs, query.dictionary_attributes_list); diff --git a/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference b/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference index d360a046958..933fda604d7 100644 --- a/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference +++ b/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference @@ -1,11 +1,11 @@ -CREATE TABLE test_01083.file\n(\n `n` Int8\n)\nENGINE = File(\'TSVWithNamesAndTypes\') -CREATE TABLE test_01083.buffer\n(\n `n` Int8\n)\nENGINE = Buffer(\'test_01083\', \'file\', 16, 10, 200, 10000, 1000000, 10000000, 1000000000) -CREATE TABLE test_01083.merge\n(\n `n` Int8\n)\nENGINE = Merge(\'test_01083\', \'distributed\') -CREATE TABLE test_01083.merge_tf AS merge(\'test_01083\', \'.*\') -CREATE TABLE test_01083.distributed\n(\n `n` Int8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'test_01083\', \'file\') -CREATE TABLE test_01083.distributed_tf AS cluster(\'test_shard_localhost\', \'test_01083\', \'buffer\') -CREATE TABLE test_01083.url\n(\n `n` UInt64,\n `col` String\n)\nENGINE = URL(\'https://localhost:8443/?query=select+n,+_table+from+test_01083.merge+format+CSV\', \'CSV\') -CREATE TABLE test_01083.rich_syntax AS remote(\'localhos{x|y|t}\', cluster(\'test_shard_localhost\', remote(\'127.0.0.{1..4}\', \'test_01083\', \'view\'))) -CREATE VIEW test_01083.view\n(\n `n` Int64\n) AS\nSELECT toInt64(n) AS n\nFROM \n(\n SELECT toString(n) AS n\n FROM test_01083.merge\n WHERE _table != \'qwerty\'\n ORDER BY _table ASC\n)\nUNION ALL\nSELECT *\nFROM test_01083.file -CREATE DICTIONARY test_01083.dict\n(\n `n` UInt64,\n `col` String DEFAULT \'42\'\n)\nPRIMARY KEY n\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9440 SECURE 1 USER \'default\' TABLE \'url\' DB \'test_01083\'))\nLIFETIME(MIN 0 MAX 1)\nLAYOUT(CACHE(SIZE_IN_CELLS 1)) +CREATE TABLE default.file\n(\n `n` Int8\n)\nENGINE = File(\'TSVWithNamesAndTypes\') +CREATE TABLE default.buffer\n(\n `n` Int8\n)\nENGINE = Buffer(\'default\', \'file\', 16, 10, 200, 10000, 1000000, 10000000, 1000000000) +CREATE TABLE default.merge\n(\n `n` Int8\n)\nENGINE = Merge(\'default\', \'distributed\') +CREATE TABLE default.merge_tf AS merge(\'default\', \'.*\') +CREATE TABLE default.distributed\n(\n `n` Int8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'default\', \'file\') +CREATE TABLE default.distributed_tf AS cluster(\'test_shard_localhost\', \'default\', \'buffer\') +CREATE TABLE default.url\n(\n `n` UInt64,\n `col` String\n)\nENGINE = URL(\'https://localhost:8443/?query=select+n,+_table+from+default.merge+format+CSV\', \'CSV\') +CREATE TABLE default.rich_syntax AS remote(\'localhos{x|y|t}\', cluster(\'test_shard_localhost\', remote(\'127.0.0.{1..4}\', \'default\', \'view\'))) +CREATE VIEW default.view\n(\n `n` Int64\n) AS\nSELECT toInt64(n) AS n\nFROM \n(\n SELECT toString(n) AS n\n FROM default.merge\n WHERE _table != \'qwerty\'\n ORDER BY _table ASC\n)\nUNION ALL\nSELECT *\nFROM default.file +CREATE DICTIONARY default.dict\n(\n `n` UInt64,\n `col` String DEFAULT \'42\'\n)\nPRIMARY KEY n\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9440 SECURE 1 USER \'default\' TABLE \'url\'))\nLIFETIME(MIN 0 MAX 1)\nLAYOUT(CACHE(SIZE_IN_CELLS 1)) 16 diff --git a/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql b/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql index 8e5f5a148a5..22aa4434f19 100644 --- a/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql +++ b/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql @@ -1,7 +1,3 @@ -DROP DATABASE IF EXISTS test_01083; -CREATE DATABASE test_01083; -USE test_01083; - CREATE TABLE file (n Int8) ENGINE = File(upper('tsv') || 'WithNames' || 'AndTypes'); CREATE TABLE buffer (n Int8) ENGINE = Buffer(currentDatabase(), file, 16, 10, 200, 10000, 1000000, 10000000, 1000000000); CREATE TABLE merge (n Int8) ENGINE = Merge('', lower('DISTRIBUTED')); @@ -28,7 +24,7 @@ CREATE VIEW view AS SELECT toInt64(n) as n FROM (SELECT toString(n) as n from me SELECT nonexistentsomething; -- { serverError 47 } CREATE DICTIONARY dict (n UInt64, col String DEFAULT '42') PRIMARY KEY n -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9440 SECURE 1 USER 'default' TABLE 'url' DB 'test_01083')) LIFETIME(1) LAYOUT(CACHE(SIZE_IN_CELLS 1)); +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9440 SECURE 1 USER 'default' TABLE 'url')) LIFETIME(1) LAYOUT(CACHE(SIZE_IN_CELLS 1)); -- dict --> url --> merge |-> distributed -> file (1) -- |-> distributed_tf -> buffer -> file (1) @@ -72,5 +68,3 @@ INSERT INTO buffer VALUES (1); -- | |-> file (1) -- |-> remote(127.0.0.2) --> ... SELECT sum(n) from rich_syntax; - -DROP DATABASE test_01083; From c260ecb58e62506fa18ad7208bed7889f4a47206 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 06:27:36 +0300 Subject: [PATCH 343/374] Remove debug output --- src/Dictionaries/ClickHouseDictionarySource.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index d9a81631fca..234cbddfc80 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -74,8 +74,6 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( , pool{is_local ? nullptr : createPool(host, port, secure, db, user, password)} , load_all_query{query_builder.composeLoadAllQuery()} { - std::cerr << "DB: " << db << "\n"; - /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). context.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0)); context = copyContextAndApplySettings(path_to_settings, context, config); From 0c75ebcba36a022d128a02a01b4da117352d1e54 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 06:28:48 +0300 Subject: [PATCH 344/374] Fix typo --- src/Dictionaries/DictionarySourceFactory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/DictionarySourceFactory.h b/src/Dictionaries/DictionarySourceFactory.h index 0a68b7d6e05..1406660dfb4 100644 --- a/src/Dictionaries/DictionarySourceFactory.h +++ b/src/Dictionaries/DictionarySourceFactory.h @@ -26,7 +26,7 @@ class DictionarySourceFactory : private boost::noncopyable public: static DictionarySourceFactory & instance(); - /// 'default_database' - the database when dictionary itself was created. + /// 'default_database' - the database where dictionary itself was created. /// It is used as default_database for ClickHouse dictionary source when no explicit database was specified. /// Does not make sense for other sources. using Creator = std::function<DictionarySourcePtr( From 145ad95e2a0776a62ecf19a67c56733bdbb8a587 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 07:23:23 +0300 Subject: [PATCH 345/374] Revert check location of Docker compose files @qoega. --- utils/check-style/check-style | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 0579dcf0c26..c51c3af619e 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -73,4 +73,4 @@ find $ROOT_PATH/{src,base,programs} -name '*.h' -or -name '*.cpp' 2>/dev/null | find $ROOT_PATH -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' -name Dockerfile -type f 2>/dev/null | xargs --no-run-if-empty -n1 echo "Please move Dockerfile to docker directory:" # There shouldn't be any docker compose files outside docker directory -find $ROOT_PATH -not -path $ROOT_PATH'/tests/testflows*' -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' -name '*compose*.yml' -type f 2>/dev/null | xargs --no-run-if-empty grep -l "version:" | xargs --no-run-if-empty -n1 echo "Please move docker compose to docker directory:" +#find $ROOT_PATH -not -path $ROOT_PATH'/tests/testflows*' -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' -name '*compose*.yml' -type f 2>/dev/null | xargs --no-run-if-empty grep -l "version:" | xargs --no-run-if-empty -n1 echo "Please move docker compose to docker directory:" From 696f77326747288f3c41df8ed2d5f0d7de6b6350 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 07:29:02 +0300 Subject: [PATCH 346/374] Remove ZooKeeper from unit tests --- .../gtest_zkutil_test_multi_exception.cpp | 229 ------------------ 1 file changed, 229 deletions(-) delete mode 100644 src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp diff --git a/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp b/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp deleted file mode 100644 index de60d382301..00000000000 --- a/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp +++ /dev/null @@ -1,229 +0,0 @@ -#include <Common/typeid_cast.h> -#include <Common/ZooKeeper/ZooKeeper.h> -#include <Common/ZooKeeper/KeeperException.h> -#include <Common/StringUtils/StringUtils.h> -#include <iostream> -#include <chrono> - -#include <gtest/gtest.h> - -#include <Common/ShellCommand.h> - - -using namespace DB; - - -template <typename... Args> -auto getZooKeeper(Args &&... args) -{ - /// In our CI infrastructure it is typical that ZooKeeper is unavailable for some amount of time. - size_t i; - for (i = 0; i < 100; ++i) - { - try - { - auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181", std::forward<Args>(args)...); - zookeeper->exists("/"); - zookeeper->createIfNotExists("/clickhouse_test", "Unit tests of ClickHouse"); - return zookeeper; - } - catch (...) - { - std::cerr << "Zookeeper is unavailable, try " << i << std::endl; - sleep(1); - continue; - } - } - - std::cerr << "No zookeeper after " << i << " tries. skip tests." << std::endl; - exit(0); -} - - -TEST(zkutil, MultiNiceExceptionMsg) -{ - auto zookeeper = getZooKeeper(); - - Coordination::Requests ops; - - ASSERT_NO_THROW( - zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi"); - - ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent)); - zookeeper->multi(ops); - ); - - try - { - ops.clear(); - ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/c", "_", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeRemoveRequest("/clickhouse_test/zkutil_multi/c", -1)); - ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "BadBoy", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/b", "_", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent)); - - zookeeper->multi(ops); - FAIL(); - } - catch (...) - { - zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi"); - - String msg = getCurrentExceptionMessage(false); - - bool msg_has_reqired_patterns = msg.find("#2") != std::string::npos; - EXPECT_TRUE(msg_has_reqired_patterns) << msg; - } -} - - -TEST(zkutil, MultiAsync) -{ - Coordination::Requests ops; - - getZooKeeper()->tryRemoveRecursive("/clickhouse_test/zkutil_multi"); - - { - ops.clear(); - auto zookeeper = getZooKeeper(); - auto fut = zookeeper->asyncMulti(ops); - } - - { - ops.clear(); - ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "", zkutil::CreateMode::Persistent)); - - auto zookeeper = getZooKeeper(); - auto fut = zookeeper->tryAsyncMulti(ops); - ops.clear(); - - auto res = fut.get(); - ASSERT_EQ(res.error, Coordination::Error::ZOK); - ASSERT_EQ(res.responses.size(), 2); - } - - EXPECT_ANY_THROW - ( - auto zookeeper = getZooKeeper(); - std::vector<std::future<Coordination::MultiResponse>> futures; - - for (size_t i = 0; i < 10000; ++i) - { - ops.clear(); - ops.emplace_back(zkutil::makeRemoveRequest("/clickhouse_test/zkutil_multi", -1)); - ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCheckRequest("/clickhouse_test/zkutil_multi", -1)); - ops.emplace_back(zkutil::makeSetRequest("/clickhouse_test/zkutil_multi", "xxx", 42)); - ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent)); - - futures.emplace_back(zookeeper->asyncMulti(ops)); - } - - futures[0].get(); - ); - - /// Check there are no segfaults for remaining 999 futures - using namespace std::chrono_literals; - std::this_thread::sleep_for(1s); - - try - { - ops.clear(); - ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent)); - - auto zookeeper = getZooKeeper(); - auto fut = zookeeper->tryAsyncMulti(ops); - ops.clear(); - - auto res = fut.get(); - - /// The test is quite heavy. It is normal if session is expired during this test. - /// If we don't check that, the test will be flacky. - if (res.error != Coordination::Error::ZSESSIONEXPIRED && res.error != Coordination::Error::ZCONNECTIONLOSS) - { - ASSERT_EQ(res.error, Coordination::Error::ZNODEEXISTS); - ASSERT_EQ(res.responses.size(), 2); - } - } - catch (const Coordination::Exception & e) - { - if (e.code != Coordination::Error::ZSESSIONEXPIRED && e.code != Coordination::Error::ZCONNECTIONLOSS) - throw; - } -} - -TEST(zkutil, WatchGetChildrenWithChroot) -{ - try - { - const String prefix = "/clickhouse_test/zkutil/watch_get_children_with_chroot"; - - /// Create chroot node firstly - auto zookeeper = getZooKeeper(); - zookeeper->createAncestors(prefix + "/"); - zookeeper = getZooKeeper("", - zkutil::DEFAULT_SESSION_TIMEOUT, - zkutil::DEFAULT_OPERATION_TIMEOUT, - prefix); - - String queue_path = "/queue"; - zookeeper->tryRemoveRecursive(queue_path); - zookeeper->createAncestors(queue_path + "/"); - - zkutil::EventPtr event = std::make_shared<Poco::Event>(); - zookeeper->getChildren(queue_path, nullptr, event); - { - auto zookeeper2 = getZooKeeper("", - zkutil::DEFAULT_SESSION_TIMEOUT, - zkutil::DEFAULT_OPERATION_TIMEOUT, - prefix); - zookeeper2->create(queue_path + "/children-", "", zkutil::CreateMode::PersistentSequential); - } - event->wait(); - } - catch (...) - { - std::cerr << getCurrentExceptionMessage(true); - throw; - } -} - -TEST(zkutil, MultiCreateSequential) -{ - try - { - const String prefix = "/clickhouse_test/zkutil"; - - /// Create chroot node firstly - auto zookeeper = getZooKeeper(); - zookeeper->createAncestors(prefix + "/"); - zookeeper = getZooKeeper("", - zkutil::DEFAULT_SESSION_TIMEOUT, - zkutil::DEFAULT_OPERATION_TIMEOUT, - "/clickhouse_test"); - - String base_path = "/multi_create_sequential"; - zookeeper->tryRemoveRecursive(base_path); - zookeeper->createAncestors(base_path + "/"); - - Coordination::Requests ops; - String sequential_node_prefix = base_path + "/queue-"; - ops.emplace_back(zkutil::makeCreateRequest(sequential_node_prefix, "", zkutil::CreateMode::EphemeralSequential)); - auto results = zookeeper->multi(ops); - const auto & sequential_node_result_op = dynamic_cast<const Coordination::CreateResponse &>(*results.at(0)); - - EXPECT_FALSE(sequential_node_result_op.path_created.empty()); - EXPECT_GT(sequential_node_result_op.path_created.length(), sequential_node_prefix.length()); - EXPECT_EQ(sequential_node_result_op.path_created.substr(0, sequential_node_prefix.length()), sequential_node_prefix); - } - catch (...) - { - std::cerr << getCurrentExceptionMessage(false); - throw; - } -} - - From ee084f0ec9936266f976286c4f759a19f0f3d10b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 08:21:02 +0300 Subject: [PATCH 347/374] Fix race condition in DETACH and background merge --- src/Storages/StorageMergeTree.cpp | 33 ++++++++++++++++--- src/Storages/StorageMergeTree.h | 2 ++ .../01060_shutdown_table_after_detach.sql | 2 +- .../01442_merge_detach_attach.reference | 0 .../0_stateless/01442_merge_detach_attach.sh | 21 ++++++++++++ .../01443_merge_truncate.reference | 0 .../0_stateless/01443_merge_truncate.sh | 21 ++++++++++++ 7 files changed, 74 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01442_merge_detach_attach.reference create mode 100755 tests/queries/0_stateless/01442_merge_detach_attach.sh create mode 100644 tests/queries/0_stateless/01443_merge_truncate.reference create mode 100755 tests/queries/0_stateless/01443_merge_truncate.sh diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index afaf6099937..64d880e6266 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -40,6 +40,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int INCORRECT_DATA; extern const int CANNOT_ASSIGN_OPTIMIZE; + extern const int TIMEOUT_EXCEEDED; } namespace ActionLocks @@ -138,7 +139,6 @@ void StorageMergeTree::shutdown() mutation_wait_event.notify_all(); } - merger_mutator.merges_blocker.cancelForever(); parts_mover.moves_blocker.cancelForever(); @@ -148,7 +148,6 @@ void StorageMergeTree::shutdown() if (moving_task_handle) global_context.getBackgroundMovePool().removeTask(moving_task_handle); - try { /// We clear all old parts after stopping all background operations. @@ -220,7 +219,7 @@ void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, cons { /// Asks to complete merges and does not allow them to start. /// This protects against "revival" of data for a removed partition after completion of merge. - auto merge_blocker = merger_mutator.merges_blocker.cancel(); + auto merge_blocker = stopMergesAndWait(); /// NOTE: It's assumed that this method is called under lockForAlter. @@ -1140,12 +1139,38 @@ Pipe StorageMergeTree::alterPartition( return {}; } + +ActionLock StorageMergeTree::stopMergesAndWait() +{ + /// Asks to complete merges and does not allow them to start. + /// This protects against "revival" of data for a removed partition after completion of merge. + auto merge_blocker = merger_mutator.merges_blocker.cancel(); + + { + std::unique_lock lock(currently_processing_in_background_mutex); + while (!currently_merging_mutating_parts.empty()) + { + LOG_DEBUG(log, "Waiting for currently running merges ({} parts are merging right now)", + currently_merging_mutating_parts.size()); + + if (std::cv_status::timeout == currently_processing_in_background_condition.wait_for( + lock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC))) + { + throw Exception("Timeout while waiting for already running merges", ErrorCodes::TIMEOUT_EXCEEDED); + } + } + } + + return merge_blocker; +} + + void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, const Context & context) { { /// Asks to complete merges and does not allow them to start. /// This protects against "revival" of data for a removed partition after completion of merge. - auto merge_blocker = merger_mutator.merges_blocker.cancel(); + auto merge_blocker = stopMergesAndWait(); auto metadata_snapshot = getInMemoryMetadataPtr(); String partition_id = getPartitionIDFromQuery(partition, context); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 66d6c64d705..5662f9e0088 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -128,6 +128,8 @@ private: */ bool merge(bool aggressive, const String & partition_id, bool final, bool deduplicate, String * out_disable_reason = nullptr); + ActionLock stopMergesAndWait(); + BackgroundProcessingPoolTaskResult movePartsTask(); /// Allocate block number for new mutation, write mutation to disk diff --git a/tests/queries/0_stateless/01060_shutdown_table_after_detach.sql b/tests/queries/0_stateless/01060_shutdown_table_after_detach.sql index 730263a2b12..1987fffaa58 100644 --- a/tests/queries/0_stateless/01060_shutdown_table_after_detach.sql +++ b/tests/queries/0_stateless/01060_shutdown_table_after_detach.sql @@ -3,7 +3,7 @@ CREATE TABLE test Engine = MergeTree ORDER BY number AS SELECT number, toString( SELECT count() FROM test; -ALTER TABLE test detach partition tuple(); +ALTER TABLE test DETACH PARTITION tuple(); SELECT count() FROM test; diff --git a/tests/queries/0_stateless/01442_merge_detach_attach.reference b/tests/queries/0_stateless/01442_merge_detach_attach.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01442_merge_detach_attach.sh b/tests/queries/0_stateless/01442_merge_detach_attach.sh new file mode 100755 index 00000000000..4577d805da5 --- /dev/null +++ b/tests/queries/0_stateless/01442_merge_detach_attach.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CUR_DIR"/../shell_config.sh + +CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE t (x Int8) ENGINE = MergeTree ORDER BY tuple()" + +for _ in {1..100}; do + ${CLICKHOUSE_CLIENT} --query="INSERT INTO t VALUES (0)" + ${CLICKHOUSE_CLIENT} --query="INSERT INTO t VALUES (0)" + ${CLICKHOUSE_CLIENT} --query="OPTIMIZE TABLE t FINAL" 2>/dev/null & + ${CLICKHOUSE_CLIENT} --query="ALTER TABLE t DETACH PARTITION tuple()" + ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM t HAVING count() > 0" +done + +wait diff --git a/tests/queries/0_stateless/01443_merge_truncate.reference b/tests/queries/0_stateless/01443_merge_truncate.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01443_merge_truncate.sh b/tests/queries/0_stateless/01443_merge_truncate.sh new file mode 100755 index 00000000000..5a0d1188ab6 --- /dev/null +++ b/tests/queries/0_stateless/01443_merge_truncate.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CUR_DIR"/../shell_config.sh + +CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE t (x Int8) ENGINE = MergeTree ORDER BY tuple()" + +for _ in {1..100}; do + ${CLICKHOUSE_CLIENT} --query="INSERT INTO t VALUES (0)" + ${CLICKHOUSE_CLIENT} --query="INSERT INTO t VALUES (0)" + ${CLICKHOUSE_CLIENT} --query="OPTIMIZE TABLE t FINAL" 2>/dev/null & + ${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE t" + ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM t HAVING count() > 0" +done + +wait From 5fe807613feef1cf5d32c9a7b3a6bd8a2d37613d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 08:37:46 +0300 Subject: [PATCH 348/374] Fix race condition in test --- .../00933_test_fix_extra_seek_on_compressed_cache.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh b/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh index 57e92c97d11..028c75b8a6c 100755 --- a/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh +++ b/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh @@ -8,7 +8,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS small_table" $CLICKHOUSE_CLIENT --query="CREATE TABLE small_table (a UInt64 default 0, n UInt64) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY (a);" -$CLICKHOUSE_CLIENT --query="INSERT INTO small_table(n) SELECT * from system.numbers limit 100000;" +$CLICKHOUSE_CLIENT --query="INSERT INTO small_table (n) SELECT * from system.numbers limit 100000;" +$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE small_table FINAL;" cached_query="SELECT count() FROM small_table where n > 0;" @@ -16,7 +17,6 @@ $CLICKHOUSE_CLIENT --use_uncompressed_cache=1 --query="$cached_query" &> /dev/nu $CLICKHOUSE_CLIENT --use_uncompressed_cache=1 --query_id="test-query-uncompressed-cache" --query="$cached_query" &> /dev/null -sleep 1 $CLICKHOUSE_CLIENT --query="SYSTEM FLUSH LOGS" $CLICKHOUSE_CLIENT --query="SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'Seek')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'ReadCompressedBytes')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'UncompressedCacheHits')] AS hit FROM system.query_log WHERE (query_id = 'test-query-uncompressed-cache') AND (type = 2) AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1" From 096209881b6f6f75bd5bebb54adf1884f5809c65 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 09:41:28 +0300 Subject: [PATCH 349/374] Fix bad test sensitive_data_masking --- .../00956_sensitive_data_masking.sh | 40 +++++++++---------- 1 file changed, 18 insertions(+), 22 deletions(-) diff --git a/tests/queries/0_stateless/00956_sensitive_data_masking.sh b/tests/queries/0_stateless/00956_sensitive_data_masking.sh index 337041be01b..9e3718b55f5 100755 --- a/tests/queries/0_stateless/00956_sensitive_data_masking.sh +++ b/tests/queries/0_stateless/00956_sensitive_data_masking.sh @@ -2,9 +2,6 @@ # Get all server logs export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL="trace" -#export CLICKHOUSE_BINARY='../../../../build-vscode/Debug/programs/clickhouse' -#export CLICKHOUSE_PORT_TCP=59000 -#export CLICKHOUSE_CLIENT_BINARY='../../../../cmake-build-debug/programs/clickhouse client' CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh @@ -20,8 +17,8 @@ $CLICKHOUSE_CLIENT \ --query="SELECT 'find_me_TOPSECRET=TOPSECRET' FROM numbers(1) FORMAT Null" \ --log_queries=1 --ignore-error --multiquery >"$tmp_file" 2>&1 -grep 'find_me_\[hidden\]' "$tmp_file" >/dev/null || echo 'fail 1a' -grep 'TOPSECRET' "$tmp_file" && echo 'fail 1b' +grep -F 'find_me_[hidden]' "$tmp_file" >/dev/null || echo 'fail 1a' +grep -F 'TOPSECRET' "$tmp_file" && echo 'fail 1b' rm -f "$tmp_file" >/dev/null 2>&1 echo 2 @@ -31,8 +28,8 @@ echo "SELECT 'find_me_TOPSECRET=TOPSECRET' FRRRROM numbers" | ${CLICKHOUSE_CURL} #cat $tmp_file ## can't be checked on client side! -# grep 'find_me_\[hidden\]' $tmp_file >/dev/null || echo 'fail 2a' -grep 'TOPSECRET' "$tmp_file" && echo 'fail 2b' +# grep -F 'find_me_[hidden]' $tmp_file >/dev/null || echo 'fail 2a' +grep -F 'TOPSECRET' "$tmp_file" && echo 'fail 2b' rm -f "$tmp_file" >/dev/null 2>&1 echo 3 @@ -41,8 +38,8 @@ $CLICKHOUSE_CLIENT \ --query="SELECT 'find_me_TOPSECRET=TOPSECRET' FROM non_existing_table FORMAT Null" \ --log_queries=1 --ignore-error --multiquery >"$tmp_file" 2>&1 -grep 'find_me_\[hidden\]' "$tmp_file" >/dev/null || echo 'fail 3a' -grep 'TOPSECRET' "$tmp_file" && echo 'fail 3b' +grep -F 'find_me_[hidden]' "$tmp_file" >/dev/null || echo 'fail 3a' +grep -F 'TOPSECRET' "$tmp_file" && echo 'fail 3b' rm -f "$tmp_file" >/dev/null 2>&1 echo 4 @@ -51,32 +48,32 @@ $CLICKHOUSE_CLIENT \ --query="SELECT 'find_me_TOPSECRET=TOPSECRET', intDiv( 100, number - 10) FROM numbers(11) FORMAT Null" \ --log_queries=1 --ignore-error --max_block_size=2 --multiquery >"$tmp_file" 2>&1 -grep 'find_me_\[hidden\]' "$tmp_file" >/dev/null || echo 'fail 4a' -grep 'TOPSECRET' "$tmp_file" && echo 'fail 4b' +grep -F 'find_me_[hidden]' "$tmp_file" >/dev/null || echo 'fail 4a' +grep -F 'TOPSECRET' "$tmp_file" && echo 'fail 4b' echo 5 # run in background rm -f "$tmp_file2" >/dev/null 2>&1 bash -c "$CLICKHOUSE_CLIENT \ - --query=\"select sleepEachRow(0.5) from numbers(4) where ignore('find_me_TOPSECRET=TOPSECRET')=0 and ignore('fwerkh_that_magic_string_make_me_unique') = 0 FORMAT Null\" \ + --query=\"select count() from system.numbers where ignore('find_me_TOPSECRET=TOPSECRET')=0 and ignore('fwerkh_that_magic_string_make_me_unique') = 0 FORMAT Null\" \ --log_queries=1 --ignore-error --multiquery >$tmp_file2 2>&1" & -sleep 0.1 - # $CLICKHOUSE_CLIENT --query='SHOW PROCESSLIST' rm -f "$tmp_file" >/dev/null 2>&1 echo '5.1' # check that executing query doesn't expose secrets in processlist $CLICKHOUSE_CLIENT --query="SHOW PROCESSLIST" --log_queries=0 >"$tmp_file" 2>&1 + +$CLICKHOUSE_CLIENT --query="KILL QUERY WHERE query LIKE '%fwerkh_that_magic_string_make_me_unique%'" > /dev/null 2>&1 wait grep 'TOPSECRET' "$tmp_file2" && echo 'fail 5d' rm -f "$tmp_file2" >/dev/null 2>&1 -grep 'fwerkh_that_magic_string_make_me_unique' "$tmp_file" >"$tmp_file2" || echo 'fail 5a' -grep 'find_me_\[hidden\]' "$tmp_file2" >/dev/null || echo 'fail 5b' -grep 'TOPSECRET' "$tmp_file" && echo 'fail 5c' +grep -F 'fwerkh_that_magic_string_make_me_unique' "$tmp_file" >"$tmp_file2" || echo 'fail 5a' +grep -F 'find_me_[hidden]' "$tmp_file2" >/dev/null || echo 'fail 5b' +grep -F 'TOPSECRET' "$tmp_file" && echo 'fail 5c' # instead of disabling send_logs_level=trace (enabled globally for that test) - redir it's output to /dev/null @@ -95,7 +92,7 @@ echo 7 # and finally querylog $CLICKHOUSE_CLIENT \ --server_logs_file=/dev/null \ - --query="select * from system.query_log where event_time>now() - 10 and query like '%TOPSECRET%';" + --query="select * from system.query_log where event_time > now() - 10 and query like '%TOPSECRET%';" rm -f "$tmp_file" >/dev/null 2>&1 @@ -108,17 +105,16 @@ insert into sensitive select number as id, toDate('2019-01-01') as date, 'abcd' select * from sensitive WHERE value1 = 'find_me_TOPSECRET=TOPSECRET' FORMAT Null; drop table sensitive;" --log_queries=1 --ignore-error --multiquery >"$tmp_file" 2>&1 -grep 'find_me_\[hidden\]' "$tmp_file" >/dev/null || echo 'fail 8a' -grep 'TOPSECRET' "$tmp_file" && echo 'fail 8b' +grep -F 'find_me_[hidden]' "$tmp_file" >/dev/null || echo 'fail 8a' +grep -F 'TOPSECRET' "$tmp_file" && echo 'fail 8b' $CLICKHOUSE_CLIENT --query="SYSTEM FLUSH LOGS" --server_logs_file=/dev/null -sleep 0.1; echo 9 $CLICKHOUSE_CLIENT \ --server_logs_file=/dev/null \ --query="SELECT if( count() > 0, 'text_log non empty', 'text_log empty') FROM system.text_log WHERE event_time>now() - 60 and message like '%find_me%'; - select * from system.text_log where event_time>now() - 60 and message like '%TOPSECRET=TOPSECRET%';" --ignore-error --multiquery + select * from system.text_log where event_time > now() - 60 and message like '%TOPSECRET=TOPSECRET%';" --ignore-error --multiquery echo 'finish' rm -f "$tmp_file" >/dev/null 2>&1 From 9e6d4802ac48639da327bdae8ae8c6759e59fd4d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 09:56:47 +0300 Subject: [PATCH 350/374] Do not drop cache if DROP/RENAME in Atomic database --- src/Storages/MergeTree/MergeTreeData.cpp | 8 ++++++-- src/Storages/StorageMergeTree.cpp | 2 +- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 173a1353f7e..bbefba70c58 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1225,7 +1225,8 @@ void MergeTreeData::rename(const String & new_table_path, const StorageID & new_ disk->moveDirectory(relative_data_path, new_table_path); } - global_context.dropCaches(); + if (!getStorageID().hasUUID()) + global_context.dropCaches(); relative_data_path = new_table_path; renameInMemory(new_table_id); @@ -1244,7 +1245,10 @@ void MergeTreeData::dropAllData() data_parts_indexes.clear(); column_sizes.clear(); - global_context.dropCaches(); + /// Tables in atomic databases have UUID and stored in persistent locations. + /// No need to drop caches (that are keyed by filesystem path) because collision is not possible. + if (!getStorageID().hasUUID()) + global_context.dropCaches(); LOG_TRACE(log, "dropAllData: removing data from filesystem."); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index afaf6099937..4937bb1a1e3 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1175,7 +1175,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons } -PartitionCommandsResultInfo StorageMergeTree::attachPartition( +PartitionCommandsResultInfo StorageMergeTree::attachPartition( const ASTPtr & partition, bool attach_part, const Context & context) { PartitionCommandsResultInfo results; From 0140093b17e38e63c5b23fc46489153b867d2a5f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 10:11:19 +0300 Subject: [PATCH 351/374] Fix fast test --- docker/test/fasttest/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index fd8a7099b60..faadfca1210 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -158,6 +158,8 @@ TESTS_TO_SKIP=( 01280_ssd_complex_key_dictionary 00652_replicated_mutations_zookeeper 01411_bayesian_ab_testing + 01238_http_memory_tracking # max_memory_usage_for_user can interfere another queries running concurrently + 01281_group_by_limit_memory_tracking # max_memory_usage_for_user can interfere another queries running concurrently ) clickhouse-test -j 4 --no-long --testname --shard --zookeeper --skip ${TESTS_TO_SKIP[*]} 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/test_log.txt From c241e42db015e990760c960a912734e529721657 Mon Sep 17 00:00:00 2001 From: alexey-milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 11:05:38 +0300 Subject: [PATCH 352/374] Update Dockerfile --- docker/packager/deb/Dockerfile | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index ac8caf2d1d0..b1f711509fb 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -54,7 +54,8 @@ RUN apt-get --allow-unauthenticated update -y \ alien \ cmake \ gdb \ - moreutils + moreutils \ + pigz # This symlink required by gcc to find lld compiler From d95b9f73a4bad774b44fcd82be67f820bde11fa6 Mon Sep 17 00:00:00 2001 From: alexey-milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 11:11:10 +0300 Subject: [PATCH 353/374] Update TabSeparatedRowInputFormat.cpp --- src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index b93d22d1f41..c394aff3f2c 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -38,7 +38,7 @@ static void checkForCarriageReturn(ReadBuffer & in) throw Exception("\nYou have carriage return (\\r, 0x0D, ASCII 13) at end of first row." "\nIt's like your input data has DOS/Windows style line separators, that are illegal in TabSeparated format." " You must transform your file to Unix format." - "\nBut if you really need carriage return at end of string value of last column, you need to escape it as \\\\r.", + "\nBut if you really need carriage return at end of string value of last column, you need to escape it as \\r.", ErrorCodes::INCORRECT_DATA); } From 1d51d9e8b63e673cc0502a50f5e8a1c8fa8fc00c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 11:49:54 +0300 Subject: [PATCH 354/374] Range check for h3KRing --- src/Functions/h3kRing.cpp | 12 ++++++++++++ .../0_stateless/01442_h3kring_range_check.reference | 1 + .../0_stateless/01442_h3kring_range_check.sql | 4 ++++ 3 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/01442_h3kring_range_check.reference create mode 100644 tests/queries/0_stateless/01442_h3kring_range_check.sql diff --git a/src/Functions/h3kRing.cpp b/src/Functions/h3kRing.cpp index 0fb9c4b6a64..9702edf7079 100644 --- a/src/Functions/h3kRing.cpp +++ b/src/Functions/h3kRing.cpp @@ -14,10 +14,13 @@ namespace DB { + namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int PARAMETER_OUT_OF_BOUND; } + class FunctionH3KRing : public IFunction { public: @@ -65,6 +68,15 @@ public: const H3Index origin_hindex = col_hindex->getUInt(row); const int k = col_k->getInt(row); + /// Overflow is possible. The function maxKringSize does not check for overflow. + /// The calculation is similar to square of k but several times more. + /// Let's use huge underestimation as the safe bound. We should not allow to generate too large arrays nevertheless. + constexpr auto max_k = 10000; + if (k > max_k) + throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Too large 'k' argument for {} function, maximum {}", getName(), max_k); + if (k < 0) + throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Argument 'k' for {} function must be non negative", getName()); + const auto vec_size = maxKringSize(k); hindex_vec.resize(vec_size); kRing(origin_hindex, k, hindex_vec.data()); diff --git a/tests/queries/0_stateless/01442_h3kring_range_check.reference b/tests/queries/0_stateless/01442_h3kring_range_check.reference new file mode 100644 index 00000000000..9f54fe3133b --- /dev/null +++ b/tests/queries/0_stateless/01442_h3kring_range_check.reference @@ -0,0 +1 @@ +122 diff --git a/tests/queries/0_stateless/01442_h3kring_range_check.sql b/tests/queries/0_stateless/01442_h3kring_range_check.sql new file mode 100644 index 00000000000..e580beb17a3 --- /dev/null +++ b/tests/queries/0_stateless/01442_h3kring_range_check.sql @@ -0,0 +1,4 @@ +SELECT h3kRing(581276613233082367, 65535); -- { serverError 12 } +SELECT h3kRing(581276613233082367, -1); -- { serverError 12 } +SELECT length(h3kRing(111111111111, 1000)); +SELECT h3kRing(581276613233082367, nan); -- { serverError 43 } From 57b8d3f89b4eb6c4ce723d68cfc72786407eed8b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 12:13:52 +0300 Subject: [PATCH 355/374] Avoid too slow queries with arrays --- src/Columns/ColumnArray.cpp | 19 +++++++++++++++++++ src/Functions/array/range.cpp | 13 +++++++++---- 2 files changed, 28 insertions(+), 4 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 7e4e6233f73..58543d6a4dd 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -31,8 +31,17 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int LOGICAL_ERROR; + extern const int TOO_LARGE_ARRAY_SIZE; } +/** Obtaining array as Field can be slow for large arrays and consume vast amount of memory. + * Just don't allow to do it. + * You can increase the limit if the following query: + * SELECT range(10000000) + * will take less than 500ms on your machine. + */ +static constexpr size_t max_array_size_as_field = 1000000; + ColumnArray::ColumnArray(MutableColumnPtr && nested_column, MutableColumnPtr && offsets_column) : data(std::move(nested_column)), offsets(std::move(offsets_column)) @@ -117,6 +126,11 @@ Field ColumnArray::operator[](size_t n) const { size_t offset = offsetAt(n); size_t size = sizeAt(n); + + if (size > max_array_size_as_field) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array of size {} is too large to be manipulated as single field, maximum size {}", + size, max_array_size_as_field); + Array res(size); for (size_t i = 0; i < size; ++i) @@ -130,6 +144,11 @@ void ColumnArray::get(size_t n, Field & res) const { size_t offset = offsetAt(n); size_t size = sizeAt(n); + + if (size > max_array_size_as_field) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array of size {} is too large to be manipulated as single field, maximum size {}", + size, max_array_size_as_field); + res = Array(size); Array & res_arr = DB::get<Array &>(res); diff --git a/src/Functions/array/range.cpp b/src/Functions/array/range.cpp index 16f95ac2682..9468b4df75d 100644 --- a/src/Functions/array/range.cpp +++ b/src/Functions/array/range.cpp @@ -102,7 +102,8 @@ private: } template <typename T> - bool executeConstStartStep(Block & block, const IColumn * end_arg, const T start, const T step, const size_t input_rows_count, const size_t result) const + bool executeConstStartStep( + Block & block, const IColumn * end_arg, const T start, const T step, const size_t input_rows_count, const size_t result) const { auto end_column = checkAndGetColumn<ColumnVector<T>>(end_arg); if (!end_column) @@ -155,7 +156,8 @@ private: } template <typename T> - bool executeConstStep(Block & block, const IColumn * start_arg, const IColumn * end_arg, const T step, const size_t input_rows_count, const size_t result) const + bool executeConstStep( + Block & block, const IColumn * start_arg, const IColumn * end_arg, const T step, const size_t input_rows_count, const size_t result) const { auto start_column = checkAndGetColumn<ColumnVector<T>>(start_arg); auto end_column = checkAndGetColumn<ColumnVector<T>>(end_arg); @@ -210,7 +212,8 @@ private: } template <typename T> - bool executeConstStart(Block & block, const IColumn * end_arg, const IColumn * step_arg, const T start, const size_t input_rows_count, const size_t result) const + bool executeConstStart( + Block & block, const IColumn * end_arg, const IColumn * step_arg, const T start, const size_t input_rows_count, const size_t result) const { auto end_column = checkAndGetColumn<ColumnVector<T>>(end_arg); auto step_column = checkAndGetColumn<ColumnVector<T>>(step_arg); @@ -265,7 +268,9 @@ private: } template <typename T> - bool executeGeneric(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t input_rows_count, const size_t result) const + bool executeGeneric( + Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, + const size_t input_rows_count, const size_t result) const { auto start_column = checkAndGetColumn<ColumnVector<T>>(start_col); auto end_column = checkAndGetColumn<ColumnVector<T>>(end_col); From aaddb6a086fd750e650d6782255d97a121fa4fa4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 12:37:05 +0300 Subject: [PATCH 356/374] Fix flaky test --- .../00988_constraints_replication_zookeeper.sql | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql b/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql index bc0e8a30670..57ec4460e79 100644 --- a/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql +++ b/tests/queries/0_stateless/00988_constraints_replication_zookeeper.sql @@ -31,6 +31,11 @@ INSERT INTO replicated_constraints1 VALUES (10, 10); INSERT INTO replicated_constraints2 VALUES (10, 10); ALTER TABLE replicated_constraints1 ADD CONSTRAINT b_constraint CHECK b > 10; + +-- Otherwise "Metadata on replica is not up to date with common metadata in Zookeeper. Cannot alter." is possible. +SYSTEM SYNC REPLICA replicated_constraints1; +SYSTEM SYNC REPLICA replicated_constraints2; + ALTER TABLE replicated_constraints2 ADD CONSTRAINT a_constraint CHECK a < 10; SYSTEM SYNC REPLICA replicated_constraints1; @@ -40,4 +45,4 @@ INSERT INTO replicated_constraints1 VALUES (10, 11); -- { serverError 469 } INSERT INTO replicated_constraints2 VALUES (9, 10); -- { serverError 469 } DROP TABLE replicated_constraints1; -DROP TABLE replicated_constraints2; \ No newline at end of file +DROP TABLE replicated_constraints2; From 88f4bad0db9950410e05690b32a2539e1405c41f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 13:03:51 +0300 Subject: [PATCH 357/374] Better diagnostics in clickhouse-test --- tests/clickhouse-test | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 857418193d7..5ba14cdb8aa 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -282,6 +282,10 @@ def run_tests_array(all_tests_with_params): if args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr: SERVER_DIED = True + if os.path.isfile(stdout_file): + print(", result:\n") + print(open(stdout_file).read()) + elif stderr: failures += 1 failures_chain += 1 From d4af19b3e763943e03725b4f2d613c8f17f17c92 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 14:22:01 +0300 Subject: [PATCH 358/374] Update test --- tests/queries/0_stateless/01042_h3_k_ring.reference | 1 - tests/queries/0_stateless/01042_h3_k_ring.sql | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01042_h3_k_ring.reference b/tests/queries/0_stateless/01042_h3_k_ring.reference index 860ddac7547..b427bcf58ac 100644 --- a/tests/queries/0_stateless/01042_h3_k_ring.reference +++ b/tests/queries/0_stateless/01042_h3_k_ring.reference @@ -1,3 +1,2 @@ [581250224954015743,581259021047037951,581267817140060159,581276613233082367,581500913605148671,581518505791193087,581764796395814911] [581276613233082367] -[581276613233082367] diff --git a/tests/queries/0_stateless/01042_h3_k_ring.sql b/tests/queries/0_stateless/01042_h3_k_ring.sql index 30bd2f58801..35335df7ff2 100644 --- a/tests/queries/0_stateless/01042_h3_k_ring.sql +++ b/tests/queries/0_stateless/01042_h3_k_ring.sql @@ -1,3 +1,3 @@ SELECT arraySort(h3kRing(581276613233082367, 1)); SELECT h3kRing(581276613233082367, 0); -SELECT h3kRing(581276613233082367, -1); +SELECT h3kRing(581276613233082367, -1); -- { serverError 12 } From 8c5e5ec1cf02659445e93015cb1e90fe2e95398f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 15:30:17 +0300 Subject: [PATCH 359/374] Fix flaky test 01305_replica_create_drop_zookeeper --- src/Storages/StorageReplicatedMergeTree.cpp | 35 ++++++++++++++------- 1 file changed, 23 insertions(+), 12 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1c0c0b1f351..b6d967906eb 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -268,16 +268,28 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( { bool is_first_replica = createTableIfNotExists(metadata_snapshot); - /// We have to check granularity on other replicas. If it's fixed we - /// must create our new replica with fixed granularity and store this - /// information in /replica/metadata. - other_replicas_fixed_granularity = checkFixedGranualrityInZookeeper(); + try + { + /// NOTE If it's the first replica, these requests to ZooKeeper look redundant, we already know everything. - checkTableStructure(zookeeper_path, metadata_snapshot); + /// We have to check granularity on other replicas. If it's fixed we + /// must create our new replica with fixed granularity and store this + /// information in /replica/metadata. + other_replicas_fixed_granularity = checkFixedGranualrityInZookeeper(); - Coordination::Stat metadata_stat; - current_zookeeper->get(zookeeper_path + "/metadata", &metadata_stat); - metadata_version = metadata_stat.version; + checkTableStructure(zookeeper_path, metadata_snapshot); + + Coordination::Stat metadata_stat; + current_zookeeper->get(zookeeper_path + "/metadata", &metadata_stat); + metadata_version = metadata_stat.version; + } + catch (Coordination::Exception & e) + { + if (!is_first_replica && e.code == Coordination::Error::ZNONODE) + throw Exception("Table " + zookeeper_path + " was suddenly removed.", ErrorCodes::ALL_REPLICAS_LOST); + else + throw; + } if (!is_first_replica) createReplica(metadata_snapshot); @@ -291,7 +303,6 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( } else { - /// In old tables this node may missing or be empty String replica_metadata; bool replica_metadata_exists = current_zookeeper->tryGet(replica_path + "/metadata", replica_metadata); @@ -758,9 +769,9 @@ void StorageReplicatedMergeTree::dropReplica(zkutil::ZooKeeperPtr zookeeper, con } -/** Verify that list of columns and table storage_settings_ptr match those specified in ZK (/ metadata). - * If not, throw an exception. - */ +/** Verify that list of columns and table storage_settings_ptr match those specified in ZK (/metadata). + * If not, throw an exception. + */ void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot) { auto zookeeper = getZooKeeper(); From 3fac908dc76acfc9b3d9f5ef25251cc98edab1c6 Mon Sep 17 00:00:00 2001 From: alexey-milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 15:43:58 +0300 Subject: [PATCH 360/374] Revert "Added separate docker image for unbundled build. Added clang-11, gcc-10 in packager image. Updated packager image to ubuntu 20.04" --- docker/images.json | 5 --- docker/packager/deb/Dockerfile | 49 +++++++++++++++++++----- docker/packager/packager | 7 +--- docker/packager/unbundled/Dockerfile | 56 ---------------------------- docker/packager/unbundled/build.sh | 18 --------- 5 files changed, 42 insertions(+), 93 deletions(-) delete mode 100644 docker/packager/unbundled/Dockerfile delete mode 100755 docker/packager/unbundled/build.sh diff --git a/docker/images.json b/docker/images.json index e9e83f7859e..0ab1688efb0 100644 --- a/docker/images.json +++ b/docker/images.json @@ -2,7 +2,6 @@ "docker/packager/deb": { "name": "yandex/clickhouse-deb-builder", "dependent": [ - "docker/packager/unbundled", "docker/test/stateless", "docker/test/stateless_with_coverage", "docker/test/stateless_pytest", @@ -16,10 +15,6 @@ "docker/test/pvs" ] }, - "docker/packager/unbundled": { - "name": "yandex/clickhouse-unbundled-builder", - "dependent": [] - }, "docker/test/coverage": { "name": "yandex/clickhouse-coverage", "dependent": [] diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index b1f711509fb..c1260b5c7ff 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -1,9 +1,9 @@ # docker build -t yandex/clickhouse-deb-builder . -FROM ubuntu:20.04 +FROM ubuntu:19.10 RUN apt-get --allow-unauthenticated update -y && apt-get install --yes wget gnupg RUN wget -O - https://apt.llvm.org/llvm-snapshot.gpg.key | apt-key add - -RUN echo "deb [trusted=yes] http://apt.llvm.org/focal/ llvm-toolchain-focal-11 main" >> /etc/apt/sources.list +RUN echo "deb [trusted=yes] http://apt.llvm.org/eoan/ llvm-toolchain-eoan-10 main" >> /etc/apt/sources.list # initial packages RUN apt-get --allow-unauthenticated update -y \ @@ -25,17 +25,13 @@ RUN curl -O https://clickhouse-builds.s3.yandex.net/utils/1/dpkg-deb RUN chmod +x dpkg-deb RUN cp dpkg-deb /usr/bin + +# Libraries from OS are only needed to test the "unbundled" build (that is not used in production). RUN apt-get --allow-unauthenticated update -y \ && env DEBIAN_FRONTEND=noninteractive \ apt-get --allow-unauthenticated install --yes --no-install-recommends \ - gcc-10 \ - g++-10 \ gcc-9 \ g++-9 \ - llvm-11 \ - clang-11 \ - lld-11 \ - clang-tidy-11 \ llvm-10 \ clang-10 \ lld-10 \ @@ -43,19 +39,54 @@ RUN apt-get --allow-unauthenticated update -y \ clang-9 \ lld-9 \ clang-tidy-9 \ + libicu-dev \ + libreadline-dev \ + gperf \ ninja-build \ perl \ pkg-config \ devscripts \ debhelper \ git \ + libc++-dev \ + libc++abi-dev \ + libboost-program-options-dev \ + libboost-system-dev \ + libboost-filesystem-dev \ + libboost-thread-dev \ + libboost-iostreams-dev \ + libboost-regex-dev \ + zlib1g-dev \ + liblz4-dev \ + libdouble-conversion-dev \ + librdkafka-dev \ + libpoconetssl62 \ + libpoco-dev \ + libgoogle-perftools-dev \ + libzstd-dev \ + libltdl-dev \ + libre2-dev \ + libjemalloc-dev \ + libmsgpack-dev \ + libcurl4-openssl-dev \ + opencl-headers \ + ocl-icd-libopencl1 \ + intel-opencl-icd \ + unixodbc-dev \ + odbcinst \ tzdata \ gperf \ alien \ + libcapnp-dev \ cmake \ gdb \ + pigz \ moreutils \ - pigz + libcctz-dev \ + libldap2-dev \ + libsasl2-dev \ + heimdal-multidev \ + libhyperscan-dev # This symlink required by gcc to find lld compiler diff --git a/docker/packager/packager b/docker/packager/packager index 63652da2439..bc97429336c 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -11,7 +11,6 @@ SCRIPT_PATH = os.path.realpath(__file__) IMAGE_MAP = { "deb": "yandex/clickhouse-deb-builder", "binary": "yandex/clickhouse-binary-builder", - "unbundled": "yandex/clickhouse-unbundled-builder" } def check_image_exists_locally(image_name): @@ -177,9 +176,7 @@ if __name__ == "__main__": parser.add_argument("--clickhouse-repo-path", default=os.path.join(os.path.dirname(os.path.abspath(__file__)), os.pardir, os.pardir)) parser.add_argument("--output-dir", required=True) parser.add_argument("--build-type", choices=("debug", ""), default="") - parser.add_argument("--compiler", choices=("clang-10", "clang-10-darwin", "clang-10-aarch64", "clang-10-freebsd", - "clang-11", "clang-11-darwin", "clang-11-aarch64", "clang-11-freebsd", - "gcc-9", "gcc-10"), default="gcc-9") + parser.add_argument("--compiler", choices=("clang-10-darwin", "clang-10-aarch64", "clang-10-freebsd", "gcc-9", "clang-10"), default="gcc-9") parser.add_argument("--sanitizer", choices=("address", "thread", "memory", "undefined", ""), default="") parser.add_argument("--unbundled", action="store_true") parser.add_argument("--split-binary", action="store_true") @@ -200,7 +197,7 @@ if __name__ == "__main__": if not os.path.isabs(args.output_dir): args.output_dir = os.path.abspath(os.path.join(os.getcwd(), args.output_dir)) - image_type = 'binary' if args.package_type == 'performance' else 'unbundled' if args.unbundled else args.package_type + image_type = 'binary' if args.package_type == 'performance' else args.package_type image_name = IMAGE_MAP[image_type] if not os.path.isabs(args.clickhouse_repo_path): diff --git a/docker/packager/unbundled/Dockerfile b/docker/packager/unbundled/Dockerfile deleted file mode 100644 index d099be6cb90..00000000000 --- a/docker/packager/unbundled/Dockerfile +++ /dev/null @@ -1,56 +0,0 @@ -# docker build -t yandex/clickhouse-unbundled-builder . -FROM yandex/clickhouse-deb-builder - -# Libraries from OS are only needed to test the "unbundled" build (that is not used in production). -RUN apt-get --allow-unauthenticated update -y \ - && env DEBIAN_FRONTEND=noninteractive \ - apt-get --allow-unauthenticated install --yes --no-install-recommends \ - libicu-dev \ - libreadline-dev \ - gperf \ - perl \ - pkg-config \ - devscripts \ - libc++-dev \ - libc++abi-dev \ - libboost-program-options-dev \ - libboost-system-dev \ - libboost-filesystem-dev \ - libboost-thread-dev \ - libboost-iostreams-dev \ - libboost-regex-dev \ - zlib1g-dev \ - liblz4-dev \ - libdouble-conversion-dev \ - librdkafka-dev \ - libpoconetssl62 \ - libpoco-dev \ - libgoogle-perftools-dev \ - libzstd-dev \ - libltdl-dev \ - libre2-dev \ - libjemalloc-dev \ - libmsgpack-dev \ - libcurl4-openssl-dev \ - opencl-headers \ - ocl-icd-libopencl1 \ - intel-opencl-icd \ - unixodbc-dev \ - odbcinst \ - tzdata \ - gperf \ - alien \ - libcapnp-dev \ - cmake \ - gdb \ - pigz \ - moreutils \ - libcctz-dev \ - libldap2-dev \ - libsasl2-dev \ - heimdal-multidev \ - libhyperscan-dev - -COPY build.sh / - -CMD ["/bin/bash", "/build.sh"] diff --git a/docker/packager/unbundled/build.sh b/docker/packager/unbundled/build.sh deleted file mode 100755 index 9fd246bcc33..00000000000 --- a/docker/packager/unbundled/build.sh +++ /dev/null @@ -1,18 +0,0 @@ -#!/usr/bin/env bash - -set -x -e - -# Update tzdata to the latest version. It is embedded into clickhouse binary. -sudo apt-get update && sudo apt-get install tzdata - -ccache --show-stats ||: -ccache --zero-stats ||: -build/release --no-pbuilder $ALIEN_PKGS | ts '%Y-%m-%d %H:%M:%S' -mv /*.deb /output -mv *.changes /output -mv *.buildinfo /output -mv /*.rpm /output ||: # if exists -mv /*.tgz /output ||: # if exists - -ccache --show-stats ||: -ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||: From cca417b058fdcf551390da8296b4307e127b3c03 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev <vladimir.chebotarev@gmail.com> Date: Sat, 15 Aug 2020 16:00:53 +0300 Subject: [PATCH 361/374] Fixed flappy `test_multiple_disks::test_start_stop_moves`. --- tests/integration/test_multiple_disks/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index d00450bf245..bb54920593f 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -531,7 +531,6 @@ def test_start_stop_moves(start_cluster, name, engine): assert used_disks[0] == 'jbod1' node1.query("SYSTEM START MOVES {}".format(name)) - node1.query("SYSTEM START MERGES {}".format(name)) # wait sometime until background backoff finishes retry = 30 @@ -541,6 +540,8 @@ def test_start_stop_moves(start_cluster, name, engine): used_disks = get_used_disks_for_table(node1, name) i += 1 + node1.query("SYSTEM START MERGES {}".format(name)) + assert sum(1 for x in used_disks if x == 'jbod1') <= 2 # first (oldest) part moved to external From 5b3fa69d698d4845b8f10fbcb115f60385e87e2b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 16:26:56 +0300 Subject: [PATCH 362/374] Make a test slightly more reliable --- tests/queries/0_stateless/01249_flush_interactive.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01249_flush_interactive.sh b/tests/queries/0_stateless/01249_flush_interactive.sh index 01b91cbace8..6049c70b9df 100755 --- a/tests/queries/0_stateless/01249_flush_interactive.sh +++ b/tests/queries/0_stateless/01249_flush_interactive.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # unless the my-program will try to output a thousand more lines overflowing pipe buffer and terminating with Broken Pipe. # But if my program just output 5 (or slightly more) lines and hang up, the pipeline is not terminated. -timeout 1 ${CLICKHOUSE_LOCAL} --max_execution_time 10 --query "SELECT DISTINCT number % 5 FROM system.numbers" ||: +timeout 5 ${CLICKHOUSE_LOCAL} --max_execution_time 10 --query "SELECT DISTINCT number % 5 FROM system.numbers" ||: echo '---' -timeout 1 ${CLICKHOUSE_CURL} -sS --no-buffer "${CLICKHOUSE_URL}&max_execution_time=10" --data-binary "SELECT DISTINCT number % 5 FROM system.numbers" ||: +timeout 5 ${CLICKHOUSE_CURL} -sS --no-buffer "${CLICKHOUSE_URL}&max_execution_time=10" --data-binary "SELECT DISTINCT number % 5 FROM system.numbers" ||: echo '---' From eb548c08b75696a1a0f3b568516c546ad3694468 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 17:35:31 +0300 Subject: [PATCH 363/374] Fix flaky mutations test --- .../0_stateless/01414_mutations_and_errors_zookeeper.sh | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh b/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh index 3e17ea205d5..1f14a891dd4 100755 --- a/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh +++ b/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh @@ -29,7 +29,7 @@ query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) while [ "$query_result" != "1" ] do query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) - sleep 0.5 + sleep 0.1 done $CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE table='replicated_mutation_table' and database='$CLICKHOUSE_DATABASE' and mutation_id='0000000000'" &> /dev/null @@ -42,9 +42,10 @@ done wait + $CLICKHOUSE_CLIENT --query "ALTER TABLE replicated_mutation_table MODIFY COLUMN value UInt64 SETTINGS replication_alter_partitions_sync = 2" 2>&1 | grep -o "Cannot parse string 'Hello' as UInt64" | head -n 1 & -check_query="SELECT count() FROM system.mutations WHERE table='replicated_mutation_table' and database='$CLICKHOUSE_DATABASE' and mutation_id='0000000001'" +check_query="SELECT type = 'UInt64' FROM system.columns WHERE table='replicated_mutation_table' and database='$CLICKHOUSE_DATABASE' and name='value'" query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) @@ -56,6 +57,9 @@ done wait + +check_query="SELECT count() FROM system.mutations WHERE table='replicated_mutation_table' and database='$CLICKHOUSE_DATABASE' and mutation_id='0000000001'" + $CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE table='replicated_mutation_table' and database='$CLICKHOUSE_DATABASE' AND mutation_id='0000000001'" &> /dev/null while [ "$query_result" != "0" ] From 4e88ba9a3e5604a551546878b480a34592f66b1e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 17:50:56 +0300 Subject: [PATCH 364/374] Fix typo --- src/Interpreters/MutationsInterpreter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 0d3ba60640b..fb6a7239d3a 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -108,7 +108,7 @@ std::optional<String> findFirstNonDeterministicFunctionName(const MutationComman ASTPtr prepareQueryAffectedAST(const std::vector<MutationCommand> & commands) { /// Execute `SELECT count() FROM storage WHERE predicate1 OR predicate2 OR ...` query. - /// The result can differ from tne number of affected rows (e.g. if there is an UPDATE command that + /// The result can differ from the number of affected rows (e.g. if there is an UPDATE command that /// changes how many rows satisfy the predicates of the subsequent commands). /// But we can be sure that if count = 0, then no rows will be touched. From 1f1685d6962fd83b86a862b463d53b49a71f5056 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 17:55:08 +0300 Subject: [PATCH 365/374] Fix error in test --- .../0_stateless/01079_parallel_alter_detach_table_zookeeper.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh b/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh index d35f40137e1..355e987a6de 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh @@ -17,6 +17,7 @@ $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_detach_1 SELECT number, $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_detach_1 SELECT number, number + 10, number from numbers(10, 40)" for i in $(seq $REPLICAS); do + $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_detach_$i" $CLICKHOUSE_CLIENT --query "SELECT SUM(value1) FROM concurrent_alter_detach_$i" done From 3a213eb785c9268f2e0283b84b7a7ad64f8d4dc7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 18:16:27 +0300 Subject: [PATCH 366/374] Keep "metadata_loading" test only for release build --- tests/queries/skip_list.json | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index c40da4c8e50..2b4d38433a7 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -12,7 +12,8 @@ "01103_check_cpu_instructions_at_startup", "01098_temporary_and_external_tables", "00152_insert_different_granularity", - "00151_replace_partition_with_different_granularity" + "00151_replace_partition_with_different_granularity", + "01193_metadata_loading" ], "address-sanitizer": [ "00281", @@ -21,7 +22,8 @@ "query_profiler", "memory_profiler", "odbc_roundtrip", - "01103_check_cpu_instructions_at_startup" + "01103_check_cpu_instructions_at_startup", + "01193_metadata_loading" ], "ub-sanitizer": [ "00281", @@ -30,7 +32,8 @@ "query_profiler", "memory_profiler", "01103_check_cpu_instructions_at_startup", - "00900_orc_load" + "00900_orc_load", + "01193_metadata_loading" ], "memory-sanitizer": [ "00281", @@ -41,7 +44,8 @@ "01103_check_cpu_instructions_at_startup", "01086_odbc_roundtrip", "00877_memory_limit_for_new_delete", - "01114_mysql_database_engine_segfault" + "01114_mysql_database_engine_segfault", + "01193_metadata_loading" ], "debug-build": [ "00281", @@ -55,7 +59,8 @@ "01200_mutations_memory_consumption", "01103_check_cpu_instructions_at_startup", "01037_polygon_dicts_", - "hyperscan" + "hyperscan", + "01193_metadata_loading" ], "unbundled-build": [ "00429", @@ -82,7 +87,8 @@ "01099_parallel_distributed_insert_select", "01300_client_save_history_when_terminated", "orc_output", - "01370_client_autocomplete_word_break_characters" + "01370_client_autocomplete_word_break_characters", + "01193_metadata_loading" ], "release-build": [ "avx2" From d25d67fcac9268d2fb5e772a9463b0c0cce4a39a Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov <vzakaznikov@protonmail.com> Date: Sat, 15 Aug 2020 13:22:46 -0400 Subject: [PATCH 367/374] Adding missing certificates and dhparam.pem for openldap4. --- tests/testflows/ldap/configs/CA/dhparam.pem | 9 ++++--- .../testflows/ldap/configs/ldap4/certs/ca.crt | 22 +++++++++++++++ .../ldap/configs/ldap4/certs/dhparam.pem | 5 ++++ .../ldap/configs/ldap4/certs/ldap.crt | 20 ++++++++++++++ .../ldap/configs/ldap4/certs/ldap.csr | 17 ++++++++++++ .../ldap/configs/ldap4/certs/ldap.key | 27 +++++++++++++++++++ .../ldap/docker-compose/docker-compose.yml | 3 --- 7 files changed, 97 insertions(+), 6 deletions(-) create mode 100644 tests/testflows/ldap/configs/ldap4/certs/ca.crt create mode 100644 tests/testflows/ldap/configs/ldap4/certs/dhparam.pem create mode 100644 tests/testflows/ldap/configs/ldap4/certs/ldap.crt create mode 100644 tests/testflows/ldap/configs/ldap4/certs/ldap.csr create mode 100644 tests/testflows/ldap/configs/ldap4/certs/ldap.key diff --git a/tests/testflows/ldap/configs/CA/dhparam.pem b/tests/testflows/ldap/configs/CA/dhparam.pem index 0a96faffd62..554d75696ee 100644 --- a/tests/testflows/ldap/configs/CA/dhparam.pem +++ b/tests/testflows/ldap/configs/CA/dhparam.pem @@ -1,5 +1,8 @@ -----BEGIN DH PARAMETERS----- -MIGHAoGBAJitt2hhnpDViQ5ko2ipBMdjy+bZ6FR/WdZ987R7lQvBkKehPXmxtEyV -AO6ofv5CZSDJokc5bUeBOAtg0EhMTCH82uPdwQvt58jRXcxXBg4JTjkx+oW9LBv2 -FdZsbaX8+SYivmiZ0Jp8T/HBm/4DA9VBS0O5GFRS4C7dHhmSTPfDAgEC +MIIBCAKCAQEA1iatTn4jdw1WIu09qeLj8OEeLhzG/w2lI4RUeJT9nU+WTwegpvLN +/MvrIMIKHRmItyxgraYFau2moC7RKm7OKLmFt6e34QeMvM1vXpuwQav6mfp8GsYL +mEIw5riFcB73E32NN3g7qmfmurkTF28BohmqhuQp2et7FNoGBKQ6ePZzGHWil3yG +nEnCwyK0o3eP2IEytx2N50uUWVdfg3MN34L3wqpUivArrjBkoMpqm3/V3wdfoYG9 +ZQkH0gIxT/2FIixCLGlfBsJ1qA/Apz1BJZbGqVu5M5iiQmq+LWN5JLS3xYai4wJL +rIY8DhjbciSNVWkwTJHzaLwIQa9a6p6mUwIBAg== -----END DH PARAMETERS----- diff --git a/tests/testflows/ldap/configs/ldap4/certs/ca.crt b/tests/testflows/ldap/configs/ldap4/certs/ca.crt new file mode 100644 index 00000000000..8c71e3afc91 --- /dev/null +++ b/tests/testflows/ldap/configs/ldap4/certs/ca.crt @@ -0,0 +1,22 @@ +-----BEGIN CERTIFICATE----- +MIIDlTCCAn2gAwIBAgIUJBqw2dHM2DDCZjYSkPOESlvDH6swDQYJKoZIhvcNAQEL +BQAwWjELMAkGA1UEBhMCQ0ExCzAJBgNVBAgMAk9OMQ8wDQYDVQQHDAZPdHRhd2Ex +ETAPBgNVBAoMCEFsdGluaXR5MQswCQYDVQQLDAJRQTENMAsGA1UEAwwEcm9vdDAe +Fw0yMDA2MTExOTAzNDhaFw0zMDA2MDkxOTAzNDhaMFoxCzAJBgNVBAYTAkNBMQsw +CQYDVQQIDAJPTjEPMA0GA1UEBwwGT3R0YXdhMREwDwYDVQQKDAhBbHRpbml0eTEL +MAkGA1UECwwCUUExDTALBgNVBAMMBHJvb3QwggEiMA0GCSqGSIb3DQEBAQUAA4IB +DwAwggEKAoIBAQC9Irr0zGV+HCI2fZ0ht4hR5It4Sbjz4RwZV8ENRP/+TEz8l9eK +J6ygxhKX7SMYzIs/jS9Gsq4plX1r2ujW1qRf8yLpR4+dGLP+jBRi1drj0XjZXosT +SERjWzgPauWxL9LN8+l26eBAqz6fw5e0W8WRSTgf5iGiCcKOTmaATIUjP0CdfWKK +qpktI4vhe++CXZFJ3usR+8KZ/FwwbCLJM/3J2HnbcXfcaYPYvr1tfqLudKSTbG9H +M3+AVwjctdesc/0sbd51Zsm0ClQptMbuKnDCYauGg61kNkgbgPgRmH9Pzo67DtxF +/WW+PtOzq8xLOifciQ9Piboy9QBSQZGwf4wzAgMBAAGjUzBRMB0GA1UdDgQWBBSi +njya0RDozx3OZTLYFpwqYnlpIDAfBgNVHSMEGDAWgBSinjya0RDozx3OZTLYFpwq +YnlpIDAPBgNVHRMBAf8EBTADAQH/MA0GCSqGSIb3DQEBCwUAA4IBAQBAD7VyFg7F +U1C25KFvtauchAOjCW6w7U/b3z1dVZvcQ88/kH1VsLUcfGixlSilUEfPTJsi7OA0 +R5BQdh2GGcjUJv4iqEFGU05KvMVmRRKn08P62+ZhJxKMxG26VzcliRZzCMkI6d0W +lFwI6nM45yeqdHVh5k4xbuJzqpbD9BtXXLI+/Ra9Fx8S9ETA3GdidpZLU5P1VLxq +UuedfqyAVWZXpr6TAURGxouRmRzul9yFzbSUex+MLEIPrstjtEwV3+tBQZJz9xAS +TVPj+Nv3LO7GCq54bdwkq1ioWbSL2hEmABkj6kdW/JwmfhGHf/2rirDVMzrTYw07 +dFJfAZC+FEsv +-----END CERTIFICATE----- diff --git a/tests/testflows/ldap/configs/ldap4/certs/dhparam.pem b/tests/testflows/ldap/configs/ldap4/certs/dhparam.pem new file mode 100644 index 00000000000..0a96faffd62 --- /dev/null +++ b/tests/testflows/ldap/configs/ldap4/certs/dhparam.pem @@ -0,0 +1,5 @@ +-----BEGIN DH PARAMETERS----- +MIGHAoGBAJitt2hhnpDViQ5ko2ipBMdjy+bZ6FR/WdZ987R7lQvBkKehPXmxtEyV +AO6ofv5CZSDJokc5bUeBOAtg0EhMTCH82uPdwQvt58jRXcxXBg4JTjkx+oW9LBv2 +FdZsbaX8+SYivmiZ0Jp8T/HBm/4DA9VBS0O5GFRS4C7dHhmSTPfDAgEC +-----END DH PARAMETERS----- diff --git a/tests/testflows/ldap/configs/ldap4/certs/ldap.crt b/tests/testflows/ldap/configs/ldap4/certs/ldap.crt new file mode 100644 index 00000000000..9167cbf861d --- /dev/null +++ b/tests/testflows/ldap/configs/ldap4/certs/ldap.crt @@ -0,0 +1,20 @@ +-----BEGIN CERTIFICATE----- +MIIDQDCCAigCFCJ7El0ntrGktZVTYTZd+OwtcJjBMA0GCSqGSIb3DQEBCwUAMFox +CzAJBgNVBAYTAkNBMQswCQYDVQQIDAJPTjEPMA0GA1UEBwwGT3R0YXdhMREwDwYD +VQQKDAhBbHRpbml0eTELMAkGA1UECwwCUUExDTALBgNVBAMMBHJvb3QwHhcNMjAw +NjExMTkxMTQzWhcNMzAwNjA5MTkxMTQzWjBfMQswCQYDVQQGEwJDQTELMAkGA1UE +CAwCT04xDzANBgNVBAcMBk90dGF3YTERMA8GA1UECgwIQWx0aW5pdHkxCzAJBgNV +BAsMAlFBMRIwEAYDVQQDDAlvcGVubGRhcDIwggEiMA0GCSqGSIb3DQEBAQUAA4IB +DwAwggEKAoIBAQC0Mbn//U56URavMgXm82FWP6vBdKuRydFX/L0M5XLlnAtk/IXG +/T+4t7nOBJxWmTp/xpsPtSMALE4eFJpEUEqlpVbG5DfBzVWcYOWoMeRAcHWCDkzr +PkB6I0dfF0Mm5hoaDhn+ZXjBWvoh/IlJdAnPg5mlejflJBQ7xtFC9eN6WjldXuRO +vyntGNuMfVLgITHwXuH2yZ98G0mFO6TU/9dRY/Z3D6RTSzKdb17Yk/VnG+ry92u2 +0sgXIBvhuJuC3ksWLArwwFoMl8DVa05D4O2H76goGdCcQ0KzqBV8RPXAh3UcgP2e +Zu90p2EGIhIk+sZTCkPd4dorxjL9nkRR86HdAgMBAAEwDQYJKoZIhvcNAQELBQAD +ggEBAJWiCxJaTksv/BTsh/etxlDY5eHwqStqIuiovEQ8bhGAcKJ3bfWd/YTb8DUS +hrLvXrXdOVC+U8PqPFXBpdOqcm5Dc233z52VgUCb+0EKv3lAzgKXRIo32h52skdK +NnRrCHDeDzgfEIXR4MEJ99cLEaxWyXQhremmTYWHYznry9/4NYz40gCDxHn9dJAi +KxFyDNxhtuKs58zp4PrBoo+542JurAoLPtRGOhdXpU2RkQVU/ho38HsAXDStAB5D +vAoSxPuMHKgo17ffrb0oqU3didwaA9fIsz7Mr6RxmI7X03s7hLzNBq9FCqu0U3RR +CX4zWGFNJu/ieSGVWLYKQzbYxp8= +-----END CERTIFICATE----- diff --git a/tests/testflows/ldap/configs/ldap4/certs/ldap.csr b/tests/testflows/ldap/configs/ldap4/certs/ldap.csr new file mode 100644 index 00000000000..bf569f727d6 --- /dev/null +++ b/tests/testflows/ldap/configs/ldap4/certs/ldap.csr @@ -0,0 +1,17 @@ +-----BEGIN CERTIFICATE REQUEST----- +MIICpDCCAYwCAQAwXzELMAkGA1UEBhMCQ0ExCzAJBgNVBAgMAk9OMQ8wDQYDVQQH +DAZPdHRhd2ExETAPBgNVBAoMCEFsdGluaXR5MQswCQYDVQQLDAJRQTESMBAGA1UE +AwwJb3BlbmxkYXAyMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAtDG5 +//1OelEWrzIF5vNhVj+rwXSrkcnRV/y9DOVy5ZwLZPyFxv0/uLe5zgScVpk6f8ab +D7UjACxOHhSaRFBKpaVWxuQ3wc1VnGDlqDHkQHB1gg5M6z5AeiNHXxdDJuYaGg4Z +/mV4wVr6IfyJSXQJz4OZpXo35SQUO8bRQvXjelo5XV7kTr8p7RjbjH1S4CEx8F7h +9smffBtJhTuk1P/XUWP2dw+kU0synW9e2JP1Zxvq8vdrttLIFyAb4bibgt5LFiwK +8MBaDJfA1WtOQ+Dth++oKBnQnENCs6gVfET1wId1HID9nmbvdKdhBiISJPrGUwpD +3eHaK8Yy/Z5EUfOh3QIDAQABoAAwDQYJKoZIhvcNAQELBQADggEBAEzIjZQOT5R7 +mEJg+RFpCSIoPn3xJ4/VMMyWqA3bTGZKpb4S6GxgsierY/87kPL7jZrMdGYB4Dc3 +2M3VWZGXlYo8vctH1zLE9VW6CzosUpl20lhdgydoCMz3RQqdJyK8aGeFTeLtk7G/ +TRCCUFUE6jaA+VtaCPCnOJSff3jUf76xguEu7dgTZgCKV7dtBqald8gIzF3D+AJJ +7pEN2UrC3UR0xpe2cj2GhndQJ+WsIyft3zpNFzAO13j8ZPibuVP7oDWcW3ixNCWC +213aeRVplJGof8Eo6llDxP+6Fwp1YmOoQmwB1Xm3t4ADn7FLJ14LONLB7q40KviG +RyLyqu3IVOI= +-----END CERTIFICATE REQUEST----- diff --git a/tests/testflows/ldap/configs/ldap4/certs/ldap.key b/tests/testflows/ldap/configs/ldap4/certs/ldap.key new file mode 100644 index 00000000000..5ab3a3f8b59 --- /dev/null +++ b/tests/testflows/ldap/configs/ldap4/certs/ldap.key @@ -0,0 +1,27 @@ +-----BEGIN RSA PRIVATE KEY----- +MIIEogIBAAKCAQEAtDG5//1OelEWrzIF5vNhVj+rwXSrkcnRV/y9DOVy5ZwLZPyF +xv0/uLe5zgScVpk6f8abD7UjACxOHhSaRFBKpaVWxuQ3wc1VnGDlqDHkQHB1gg5M +6z5AeiNHXxdDJuYaGg4Z/mV4wVr6IfyJSXQJz4OZpXo35SQUO8bRQvXjelo5XV7k +Tr8p7RjbjH1S4CEx8F7h9smffBtJhTuk1P/XUWP2dw+kU0synW9e2JP1Zxvq8vdr +ttLIFyAb4bibgt5LFiwK8MBaDJfA1WtOQ+Dth++oKBnQnENCs6gVfET1wId1HID9 +nmbvdKdhBiISJPrGUwpD3eHaK8Yy/Z5EUfOh3QIDAQABAoIBADugMMIKWcuTxYPX +c6iGZHEbxIPRTWyCcalB0nTQAAMGbabPAJ1l8432DZ+kWu806OybFXhPIfPOtVKy +0pFEWE8TtPE/V0vj3C5Qye2sBLFmBRwyCzXUdZV00wseMXRPs9dnTyalAR5KMnbI +j80kfpKSI2dkV9aU57UYBuq3Xrx/TCGItwL769D4ZZW9BvbpiTZApQQFZ0gwUFFn +btPXGU9Ti8H4mfBuZWL+5CaZdqOo76+CXvMPaUK0F9MJp4yX3XxQLRNH3qz/Tyn7 +h7QOOo0XTqoUmzRw0N9QRVH5LRdSE5yq3aF9aFKjNW59exz+62pufOFadngzkpkn +OKCzgWkCgYEA4mOWWMzdYwMn3GtfG7whqlqy7wOmMkNb81zTDQejHBV98dnj0AHr +deurfKWzHrAh3DXo6tFeqUIgXabhBPS/0dEx/S5sgLFmuUZP05EUYahfWBgzzmM9 +C6Oe5xIMLzxsZCJczolsfkEsoFe4o0vkvuLYoQrQL7InzewcDy8cUxsCgYEAy8Na +YCnanSNDY03Bulcni+5sF+opaHseeki1pv3nlw8TwsWuZF9ApS+yL7ck9jJjxBRR +RC3KGmpoqIr0vTmUYS946ngQWXPE90zfuhJfM+NRv/q0oCjH0qAcxRbTkls5On9v +oxJ8rO7gD6K85eHqasWdbCVzdZrobOXzay37tmcCgYBfyUUmw190cjReZauzH3Gb +E48b5A5gu/Fe0cqWe8G+szU7rDZgnz9SAGnpbm6QMHPTKZgoKngD42+wUFhq8Wdr +zjh5aDgOZ4EQKTjDSmI2Q7g7nNnmnESK9SrZl+BB6C3wXD2qQaj+7nKEUTlVFlpt +jaucz+dwFtASp7Djl8pDOwKBgEtr2c3ycArt/ImLRIP2spqm+7e2YvFbcSKOOz6+ +iLRvTj8v8KcSYtlB2FC1F6dRa4AujQ4RbNduP6LzHDfWUkfOzJDtNBAIPAXVnJJB +LqAEKkRHRghqT9x0i3GgS1vHDF3MwcO4mhFgserXr9ffUWeIEgbvrdcAKbv1Oa6Y +bK1NAoGAGPm8ISmboDJynjBl9wMrkcy23Pwg9kmyocdWUHh0zMLDKriZNKYB6u/U +C+/RTfkohPoHPzkeqWiHp7z3JhMItYUfTkNW6vMCxEGc0NEN6ZyMIjtiDPGN1n6O +E7jmODFmj1AQICQGdV5SHp+yKvKyb0YHKyDwETbs4SZBXxVvjEw= +-----END RSA PRIVATE KEY----- diff --git a/tests/testflows/ldap/docker-compose/docker-compose.yml b/tests/testflows/ldap/docker-compose/docker-compose.yml index c788e089b4f..c8ff683df58 100644 --- a/tests/testflows/ldap/docker-compose/docker-compose.yml +++ b/tests/testflows/ldap/docker-compose/docker-compose.yml @@ -57,9 +57,6 @@ services: LDAP_TLS_ENFORCE: "false" LDAP_TLS_VERIFY_CLIENT: "never" LDAP_TLS_CIPHER_SUITE: "SECURE256:+SECURE128:-VERS-TLS-ALL:+VERS-TLS1.2:-RSA:-DHE-DSS:-CAMELLIA-128-CBC:-CAMELLIA-256-CBC" - healthcheck: - # May need some time to generate DH parameters... - start_period: 300s volumes: - "${CLICKHOUSE_TESTS_DIR}/configs/ldap4/config:/container/service/slapd/assets/config/bootstrap/ldif/custom" - "${CLICKHOUSE_TESTS_DIR}/configs/ldap4/certs:/container/service/slapd/assets/certs/" From 0217de37b9cfba7dbb81e63a402d41e4a8df67d6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat, 15 Aug 2020 20:55:49 +0300 Subject: [PATCH 368/374] Fix topK test --- tests/queries/0_stateless/00981_topK_topKWeighted_long.sql | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00981_topK_topKWeighted_long.sql b/tests/queries/0_stateless/00981_topK_topKWeighted_long.sql index c1f385fae80..5758f8ab8e5 100644 --- a/tests/queries/0_stateless/00981_topK_topKWeighted_long.sql +++ b/tests/queries/0_stateless/00981_topK_topKWeighted_long.sql @@ -2,8 +2,7 @@ DROP TABLE IF EXISTS topk; CREATE TABLE topk (val1 String, val2 UInt32) ENGINE = MergeTree ORDER BY val1; -INSERT INTO topk SELECT toString(number), number FROM numbers(3000000); -INSERT INTO topk SELECT toString(number % 10), 999999999 FROM numbers(1000000); +INSERT INTO topk WITH number % 7 = 0 AS frequent SELECT toString(frequent ? number % 10 : number), frequent ? 999999999 : number FROM numbers(4000000); SELECT arraySort(topK(10)(val1)) FROM topk; SELECT arraySort(topKWeighted(10)(val1, val2)) FROM topk; From 6a517be3b21fb26461292a16db7edf71c95d0f59 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov <vzakaznikov@protonmail.com> Date: Sat, 15 Aug 2020 14:24:33 -0400 Subject: [PATCH 369/374] * Moving to standard healthcheck parameters. * Adding docker-compose pull and docker-compose down before executing docker-compose up. * Adding docker-compose ps if any of the containers are unhealthy. --- tests/testflows/helpers/cluster.py | 38 +++++++++++-------- .../docker-compose/clickhouse-service.yml | 4 +- .../ldap/docker-compose/openldap-service.yml | 8 ++-- .../ldap/docker-compose/zookeeper-service.yml | 4 +- 4 files changed, 30 insertions(+), 24 deletions(-) diff --git a/tests/testflows/helpers/cluster.py b/tests/testflows/helpers/cluster.py index 660e831ffb8..e7fbc6bde01 100644 --- a/tests/testflows/helpers/cluster.py +++ b/tests/testflows/helpers/cluster.py @@ -92,7 +92,7 @@ class ClickHouseNode(Node): name, value = setting command += f" --{name} \"{value}\"" description = f""" - echo -e \"{sql[:100]}...\" > {query.name} + echo -e \"{sql[:100]}...\" > {query.name} {command} """ with step("executing command", description=description) if steps else NullStep(): @@ -128,12 +128,12 @@ class ClickHouseNode(Node): class Cluster(object): """Simple object around docker-compose cluster. """ - def __init__(self, local=False, - clickhouse_binary_path=None, configs_dir=None, + def __init__(self, local=False, + clickhouse_binary_path=None, configs_dir=None, nodes=None, - docker_compose="docker-compose", docker_compose_project_dir=None, + docker_compose="docker-compose", docker_compose_project_dir=None, docker_compose_file="docker-compose.yml"): - + self._bash = {} self.clickhouse_binary_path = clickhouse_binary_path self.configs_dir = configs_dir @@ -159,7 +159,7 @@ class Cluster(object): if os.path.exists(caller_project_dir): docker_compose_project_dir = caller_project_dir - docker_compose_file_path = os.path.join(docker_compose_project_dir or "", docker_compose_file) + docker_compose_file_path = os.path.join(docker_compose_project_dir or "", docker_compose_file) if not os.path.exists(docker_compose_file_path): raise TypeError("docker compose file '{docker_compose_file_path}' does not exist") @@ -237,7 +237,7 @@ class Cluster(object): finally: return self.command(None, f"{self.docker_compose} down", timeout=timeout) - def up(self): + def up(self, timeout=30*60): if self.local: with Given("I am running in local mode"): with Then("check --clickhouse-binary-path is specified"): @@ -245,20 +245,26 @@ class Cluster(object): with And("path should exist"): assert os.path.exists(self.clickhouse_binary_path) - os.environ["CLICKHOUSE_TESTS_SERVER_BIN_PATH"] = self.clickhouse_binary_path - os.environ["CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH"] = os.path.join( - os.path.dirname(self.clickhouse_binary_path), "clickhouse-odbc-bridge") - os.environ["CLICKHOUSE_TESTS_DIR"] = self.configs_dir + with And("I set all the necessary environment variables"): + os.environ["CLICKHOUSE_TESTS_SERVER_BIN_PATH"] = self.clickhouse_binary_path + os.environ["CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH"] = os.path.join( + os.path.dirname(self.clickhouse_binary_path), "clickhouse-odbc-bridge") + os.environ["CLICKHOUSE_TESTS_DIR"] = self.configs_dir - with Given("docker-compose"): + with And("I list environment variables to show their values"): self.command(None, "env | grep CLICKHOUSE") - cmd = self.command(None, f'{self.docker_compose} up -d 2>&1 | tee', timeout=30 * 60) - else: - with Given("docker-compose"): - cmd = self.command(None, f'{self.docker_compose} up -d --no-recreate 2>&1 | tee') + + with Given("docker-compose"): + with By("pulling images for all the services"): + self.command(None, f'{self.docker_compose} pull 2>&1 | tee', exitcode=0, timeout=timeout) + with And("executing docker-compose down just in case it is up"): + self.command(None, f'{self.docker_compose} down 2>&1 | tee', exitcode=0, timeout=timeout) + with And("executing docker-compose up"): + cmd = self.command(None, f'{self.docker_compose} up -d 2>&1 | tee', timeout=timeout) with Then("check there are no unhealthy containers"): if "is unhealthy" in cmd.output: + self.command(None, f'{self.docker_compose} ps | tee') self.command(None, f'{self.docker_compose} logs | tee') fail("found unhealthy containers") diff --git a/tests/testflows/ldap/docker-compose/clickhouse-service.yml b/tests/testflows/ldap/docker-compose/clickhouse-service.yml index 12553d7948d..5bb4995a4c0 100644 --- a/tests/testflows/ldap/docker-compose/clickhouse-service.yml +++ b/tests/testflows/ldap/docker-compose/clickhouse-service.yml @@ -20,8 +20,8 @@ services: test: clickhouse client --query='select 1' interval: 3s timeout: 10s - retries: 100 - start_period: 30s + retries: 60 + start_period: 2s cap_add: - SYS_PTRACE security_opt: diff --git a/tests/testflows/ldap/docker-compose/openldap-service.yml b/tests/testflows/ldap/docker-compose/openldap-service.yml index d642c535f30..f3ea04733d7 100644 --- a/tests/testflows/ldap/docker-compose/openldap-service.yml +++ b/tests/testflows/ldap/docker-compose/openldap-service.yml @@ -16,8 +16,8 @@ services: test: ldapsearch -x -H ldap://localhost:$${LDAP_PORT:-389} -b "dc=company,dc=com" -D "cn=admin,dc=company,dc=com" -w admin interval: 3s timeout: 10s - retries: 5 - start_period: 30s + retries: 60 + start_period: 2s security_opt: - label:disable @@ -32,8 +32,8 @@ services: healthcheck: test: echo 1 interval: 3s - timeout: 2s - retries: 5 + timeout: 10s + retries: 60 start_period: 2s security_opt: - label:disable diff --git a/tests/testflows/ldap/docker-compose/zookeeper-service.yml b/tests/testflows/ldap/docker-compose/zookeeper-service.yml index f3df33358be..60b18e0e98b 100644 --- a/tests/testflows/ldap/docker-compose/zookeeper-service.yml +++ b/tests/testflows/ldap/docker-compose/zookeeper-service.yml @@ -11,8 +11,8 @@ services: healthcheck: test: echo stat | nc localhost 2181 interval: 3s - timeout: 2s - retries: 5 + timeout: 10s + retries: 60 start_period: 2s security_opt: - label:disable From e9d373b5b7ecea0321b29596b3cf0fd17371348f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin <a3at.mail@gmail.com> Date: Sun, 16 Aug 2020 00:14:15 +0300 Subject: [PATCH 370/374] Fix step overflow in range() --- src/Functions/array/range.cpp | 24 +++++++++++++++++++ .../01408_range_overflow.reference | 0 .../0_stateless/01408_range_overflow.sql | 12 ++++++++++ 3 files changed, 36 insertions(+) create mode 100644 tests/queries/0_stateless/01408_range_overflow.reference create mode 100644 tests/queries/0_stateless/01408_range_overflow.sql diff --git a/src/Functions/array/range.cpp b/src/Functions/array/range.cpp index 16f95ac2682..7b173ea9385 100644 --- a/src/Functions/array/range.cpp +++ b/src/Functions/array/range.cpp @@ -145,8 +145,14 @@ private: for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) { for (size_t st = start, ed = end_data[row_idx]; st < ed; st += step) + { out_data[offset++] = st; + if (st > st + step) + throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + } + out_offsets[row_idx] = offset; } @@ -200,8 +206,14 @@ private: for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) { for (size_t st = start_data[row_idx], ed = end_data[row_idx]; st < ed; st += step) + { out_data[offset++] = st; + if (st > st + step) + throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + } + out_offsets[row_idx] = offset; } @@ -255,8 +267,14 @@ private: for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) { for (size_t st = start, ed = end_data[row_idx]; st < ed; st += step_data[row_idx]) + { out_data[offset++] = st; + if (st > st + step_data[row_idx]) + throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + } + out_offsets[row_idx] = offset; } @@ -313,8 +331,14 @@ private: for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) { for (size_t st = start_data[row_idx], ed = end_start[row_idx]; st < ed; st += step_data[row_idx]) + { out_data[offset++] = st; + if (st > st + step_data[row_idx]) + throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + } + out_offsets[row_idx] = offset; } diff --git a/tests/queries/0_stateless/01408_range_overflow.reference b/tests/queries/0_stateless/01408_range_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01408_range_overflow.sql b/tests/queries/0_stateless/01408_range_overflow.sql new file mode 100644 index 00000000000..1640798999c --- /dev/null +++ b/tests/queries/0_stateless/01408_range_overflow.sql @@ -0,0 +1,12 @@ +-- executeGeneric() +SELECT range(1025, 1048576 + 9223372036854775807, 9223372036854775807); -- { serverError 69; } +SELECT range(1025, 1048576 + (9223372036854775807 AS i), i); -- { serverError 69; } + +-- executeConstStep() +SELECT range(number, 1048576 + 9223372036854775807, 9223372036854775807) FROM system.numbers LIMIT 1 OFFSET 1025; -- { serverError 69; } + +-- executeConstStartStep() +SELECT range(1025, number + 9223372036854775807, 9223372036854775807) FROM system.numbers LIMIT 1 OFFSET 1048576; -- { serverError 69; } + +-- executeConstStart() +SELECT range(1025, 1048576 + 9223372036854775807, number + 9223372036854775807) FROM system.numbers LIMIT 1; -- { serverError 69; } From dfb7cb999e3eafc285691e1907b15a4c3077b1f4 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov <vzakaznikov@protonmail.com> Date: Sat, 15 Aug 2020 19:52:27 -0400 Subject: [PATCH 371/374] Updating healthcheck parameters. --- .../example/docker-compose/clickhouse-service.yml | 8 ++++---- .../example/docker-compose/zookeeper-service.yml | 8 ++++---- .../ldap/docker-compose/clickhouse-service.yml | 6 +++--- .../ldap/docker-compose/openldap-service.yml | 12 ++++++------ .../ldap/docker-compose/zookeeper-service.yml | 6 +++--- .../rbac/docker-compose/clickhouse-service.yml | 8 ++++---- .../rbac/docker-compose/zookeeper-service.yml | 8 ++++---- 7 files changed, 28 insertions(+), 28 deletions(-) diff --git a/tests/testflows/example/docker-compose/clickhouse-service.yml b/tests/testflows/example/docker-compose/clickhouse-service.yml index 36f13e40c5c..2a56876c72e 100644 --- a/tests/testflows/example/docker-compose/clickhouse-service.yml +++ b/tests/testflows/example/docker-compose/clickhouse-service.yml @@ -18,10 +18,10 @@ services: entrypoint: bash -c "clickhouse server --config-file=/etc/clickhouse-server/config.xml --log-file=/var/log/clickhouse-server/clickhouse-server.log --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log" healthcheck: test: clickhouse client --query='select 1' - interval: 3s - timeout: 5s - retries: 100 - start_period: 2s + interval: 10s + timeout: 10s + retries: 3 + start_period: 300s cap_add: - SYS_PTRACE security_opt: diff --git a/tests/testflows/example/docker-compose/zookeeper-service.yml b/tests/testflows/example/docker-compose/zookeeper-service.yml index f3df33358be..6691a2df31c 100644 --- a/tests/testflows/example/docker-compose/zookeeper-service.yml +++ b/tests/testflows/example/docker-compose/zookeeper-service.yml @@ -10,9 +10,9 @@ services: ZOO_MY_ID: 1 healthcheck: test: echo stat | nc localhost 2181 - interval: 3s - timeout: 2s - retries: 5 - start_period: 2s + interval: 10s + timeout: 10s + retries: 3 + start_period: 300s security_opt: - label:disable diff --git a/tests/testflows/ldap/docker-compose/clickhouse-service.yml b/tests/testflows/ldap/docker-compose/clickhouse-service.yml index 5bb4995a4c0..2a56876c72e 100644 --- a/tests/testflows/ldap/docker-compose/clickhouse-service.yml +++ b/tests/testflows/ldap/docker-compose/clickhouse-service.yml @@ -18,10 +18,10 @@ services: entrypoint: bash -c "clickhouse server --config-file=/etc/clickhouse-server/config.xml --log-file=/var/log/clickhouse-server/clickhouse-server.log --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log" healthcheck: test: clickhouse client --query='select 1' - interval: 3s + interval: 10s timeout: 10s - retries: 60 - start_period: 2s + retries: 3 + start_period: 300s cap_add: - SYS_PTRACE security_opt: diff --git a/tests/testflows/ldap/docker-compose/openldap-service.yml b/tests/testflows/ldap/docker-compose/openldap-service.yml index f3ea04733d7..139907c513c 100644 --- a/tests/testflows/ldap/docker-compose/openldap-service.yml +++ b/tests/testflows/ldap/docker-compose/openldap-service.yml @@ -14,10 +14,10 @@ services: - "636" healthcheck: test: ldapsearch -x -H ldap://localhost:$${LDAP_PORT:-389} -b "dc=company,dc=com" -D "cn=admin,dc=company,dc=com" -w admin - interval: 3s + interval: 10s timeout: 10s - retries: 60 - start_period: 2s + retries: 3 + start_period: 300s security_opt: - label:disable @@ -31,10 +31,10 @@ services: - "8080:80" healthcheck: test: echo 1 - interval: 3s + interval: 10s timeout: 10s - retries: 60 - start_period: 2s + retries: 3 + start_period: 300s security_opt: - label:disable diff --git a/tests/testflows/ldap/docker-compose/zookeeper-service.yml b/tests/testflows/ldap/docker-compose/zookeeper-service.yml index 60b18e0e98b..6691a2df31c 100644 --- a/tests/testflows/ldap/docker-compose/zookeeper-service.yml +++ b/tests/testflows/ldap/docker-compose/zookeeper-service.yml @@ -10,9 +10,9 @@ services: ZOO_MY_ID: 1 healthcheck: test: echo stat | nc localhost 2181 - interval: 3s + interval: 10s timeout: 10s - retries: 60 - start_period: 2s + retries: 3 + start_period: 300s security_opt: - label:disable diff --git a/tests/testflows/rbac/docker-compose/clickhouse-service.yml b/tests/testflows/rbac/docker-compose/clickhouse-service.yml index a41f0da2839..2d79443dcbb 100644 --- a/tests/testflows/rbac/docker-compose/clickhouse-service.yml +++ b/tests/testflows/rbac/docker-compose/clickhouse-service.yml @@ -18,10 +18,10 @@ services: entrypoint: bash -c "clickhouse server --config-file=/etc/clickhouse-server/config.xml --log-file=/var/log/clickhouse-server/clickhouse-server.log --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log" healthcheck: test: clickhouse client --query='select 1' - interval: 3s - timeout: 2s - retries: 100 - start_period: 2s + interval: 10s + timeout: 10s + retries: 3 + start_period: 300s cap_add: - SYS_PTRACE security_opt: diff --git a/tests/testflows/rbac/docker-compose/zookeeper-service.yml b/tests/testflows/rbac/docker-compose/zookeeper-service.yml index f3df33358be..6691a2df31c 100644 --- a/tests/testflows/rbac/docker-compose/zookeeper-service.yml +++ b/tests/testflows/rbac/docker-compose/zookeeper-service.yml @@ -10,9 +10,9 @@ services: ZOO_MY_ID: 1 healthcheck: test: echo stat | nc localhost 2181 - interval: 3s - timeout: 2s - retries: 5 - start_period: 2s + interval: 10s + timeout: 10s + retries: 3 + start_period: 300s security_opt: - label:disable From 1b929ec54af63500cfd788846d80260239fb0ec1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sun, 16 Aug 2020 11:07:36 +0300 Subject: [PATCH 372/374] Fix flaky unit test --- .../gtest_thread_pool_schedule_exception.cpp | 24 +++++++++++++++---- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/src/Common/tests/gtest_thread_pool_schedule_exception.cpp b/src/Common/tests/gtest_thread_pool_schedule_exception.cpp index 38f9365ca7f..2dad0be3867 100644 --- a/src/Common/tests/gtest_thread_pool_schedule_exception.cpp +++ b/src/Common/tests/gtest_thread_pool_schedule_exception.cpp @@ -1,6 +1,7 @@ #include <iostream> #include <stdexcept> #include <Common/ThreadPool.h> +#include <Poco/Event.h> #include <gtest/gtest.h> @@ -9,21 +10,34 @@ static bool check() { ThreadPool pool(10); + /// The throwing thread. pool.scheduleOrThrowOnError([] { throw std::runtime_error("Hello, world!"); }); try { - for (size_t i = 0; i < 500; ++i) - pool.scheduleOrThrowOnError([] {}); /// An exception will be rethrown from this method. + while (true) + { + /// An exception from the throwing thread will be rethrown from this method + /// as soon as the throwing thread executed. + + /// This innocent thread may or may not be executed, the following possibilities exist: + /// 1. The throwing thread has already throwed exception and the attempt to schedule the innocent thread will rethrow it. + /// 2. The throwing thread has not executed, the innocent thread will be scheduled and executed. + /// 3. The throwing thread has not executed, the innocent thread will be scheduled but before it will be executed, + /// the throwing thread will be executed and throw exception and it will prevent starting of execution of the innocent thread + /// the method will return and the exception will be rethrown only on call to "wait" or on next call on next loop iteration as (1). + pool.scheduleOrThrowOnError([]{}); + + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + } } catch (const std::runtime_error &) { + pool.wait(); return true; } - pool.wait(); - - return false; + __builtin_unreachable(); } From 751cce6dfc70e1c007d02877ccf14341543d2bb0 Mon Sep 17 00:00:00 2001 From: alexey-milovidov <milovidov@yandex-team.ru> Date: Sun, 16 Aug 2020 11:10:36 +0300 Subject: [PATCH 373/374] Update gtest_thread_pool_schedule_exception.cpp --- src/Common/tests/gtest_thread_pool_schedule_exception.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/tests/gtest_thread_pool_schedule_exception.cpp b/src/Common/tests/gtest_thread_pool_schedule_exception.cpp index 2dad0be3867..73fdce1cad8 100644 --- a/src/Common/tests/gtest_thread_pool_schedule_exception.cpp +++ b/src/Common/tests/gtest_thread_pool_schedule_exception.cpp @@ -1,7 +1,6 @@ #include <iostream> #include <stdexcept> #include <Common/ThreadPool.h> -#include <Poco/Event.h> #include <gtest/gtest.h> From 44fb1ec107fe5d1f68f476b704d6b431d42dc782 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sun, 16 Aug 2020 13:15:35 +0300 Subject: [PATCH 374/374] Fix typo --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b6d967906eb..cac2219b062 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -406,7 +406,7 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( } /// It maybe already removed from zk, but local in-memory mutations - /// state was not update. + /// state was not updated. if (!getZooKeeper()->exists(zookeeper_path + "/mutations/" + mutation_id)) { throw Exception(ErrorCodes::UNFINISHED, "Mutation {} was killed, manually removed or table was dropped", mutation_id);